xiarixiaoyao commented on a change in pull request #3203:
URL: https://github.com/apache/hudi/pull/3203#discussion_r725419094



##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergedLogReader.java
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.hadoop.realtime;
+
+import java.io.IOException;
+import java.text.MessageFormat;
+import java.util.Iterator;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Record Reader implementation to read avro data, to support inc queries.
+ */
+public class HoodieMergedLogReader extends AbstractRealtimeRecordReader
+    implements RecordReader<NullWritable, ArrayWritable> {
+  private static final Logger LOG = 
LogManager.getLogger(AbstractRealtimeRecordReader.class);
+  private final HoodieMergedLogRecordScanner logRecordScanner;
+  private final Iterator<HoodieRecord<? extends HoodieRecordPayload>> 
logRecordsKeyIterator;
+  private ArrayWritable valueObj;
+
+  private int end;
+  private int offset;
+
+  public HoodieMergedLogReader(RealtimeSplit split, JobConf job, 
HoodieMergedLogRecordScanner logRecordScanner) {
+    super(split, job);
+    this.logRecordScanner = logRecordScanner;
+    this.end = logRecordScanner.getRecords().size();
+    this.logRecordsKeyIterator = logRecordScanner.iterator();
+    this.valueObj = new ArrayWritable(Writable.class, new 
Writable[getHiveSchema().getFields().size()]);
+  }
+
+  private Option buildGenericRecord(HoodieRecord record) throws IOException {
+    if (usesCustomPayload) {
+      return record.getData().getInsertValue(getWriterSchema());
+    } else {
+      return record.getData().getInsertValue(getReaderSchema());
+    }
+  }
+
+  @Override
+  public boolean next(NullWritable key, ArrayWritable arrayWritable) throws 
IOException {
+    if (!logRecordsKeyIterator.hasNext()) {
+      return false;
+    }
+    Option<GenericRecord> rec;
+    HoodieRecord currentRecord = logRecordsKeyIterator.next();
+
+    rec = buildGenericRecord(currentRecord);
+    // try to skip delete record
+    while (!rec.isPresent() && logRecordsKeyIterator.hasNext()) {
+      offset++;
+      rec = buildGenericRecord(logRecordsKeyIterator.next());
+    }
+    if (!rec.isPresent()) {
+      return false;
+    }
+
+    GenericRecord recordToReturn = rec.get();
+    if (usesCustomPayload) {
+      // If using a custom payload, return only the projection fields. The 
readerSchema is a schema derived from

Review comment:
       yes, keep the logical continuity with HoodieMergedLogRecordScanner。 when 
we build HoodieMergedLogRecordScanner we use usesCustomPayload to deternime the 
read schema for log file, see follow code from RealTimeCompactedRecordReader。
       _return HoodieMergedLogRecordScanner.newBuilder()
           .withFileSystem(FSUtils.getFs(split.getPath().toString(), jobConf))
           .withBasePath(split.getBasePath())
           .withLogFilePaths(split.getDeltaLogPaths())
           .withReaderSchema(**usesCustomPayload ? getWriterSchema() : 
getReaderSchema()**)_
   
   of course, Due to the schema evolution ability of avro, use 
getWriterSchema() or getReaderSchema() is not different。




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to