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



##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergedLogReader.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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 boolean supportTimeStamp = false;
+
+  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 buildGenericRecordwithCustomPayload(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 = buildGenericRecordwithCustomPayload(currentRecord);
+    // try to skip delete record
+    while (!rec.isPresent() && logRecordsKeyIterator.hasNext()) {
+      offset++;
+      rec = buildGenericRecordwithCustomPayload(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
+      // the writerSchema with only the projection fields
+      recordToReturn = HoodieAvroUtils.rewriteRecord(rec.get(), 
getReaderSchema());
+    }
+    ArrayWritable curWritable = (ArrayWritable) 
HoodieRealtimeRecordReaderUtils.avroToArrayWritable(recordToReturn, 
getHiveSchema());
+
+    if (arrayWritable != curWritable) {
+      final Writable[] arrValue = arrayWritable.get();
+      final Writable[] arrCurrent = curWritable.get();
+      if (arrayWritable != null && arrValue.length == arrCurrent.length) {
+        System.arraycopy(arrCurrent, 0, arrValue, 0, arrCurrent.length);

Review comment:
       `arrayWritable != null` should never reach because `arrayWritable.get()` 
is invoked.

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergedLogReader.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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 boolean supportTimeStamp = false;
+
+  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 buildGenericRecordwithCustomPayload(HoodieRecord record) 
throws IOException {
+    if (usesCustomPayload) {

Review comment:
       buildGenericRecord

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergedLogReader.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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 boolean supportTimeStamp = false;
+
+  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 buildGenericRecordwithCustomPayload(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 = buildGenericRecordwithCustomPayload(currentRecord);
+    // try to skip delete record
+    while (!rec.isPresent() && logRecordsKeyIterator.hasNext()) {
+      offset++;
+      rec = buildGenericRecordwithCustomPayload(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
+      // the writerSchema with only the projection fields
+      recordToReturn = HoodieAvroUtils.rewriteRecord(rec.get(), 
getReaderSchema());
+    }
+    ArrayWritable curWritable = (ArrayWritable) 
HoodieRealtimeRecordReaderUtils.avroToArrayWritable(recordToReturn, 
getHiveSchema());
+
+    if (arrayWritable != curWritable) {
+      final Writable[] arrValue = arrayWritable.get();
+      final Writable[] arrCurrent = curWritable.get();
+      if (arrayWritable != null && arrValue.length == arrCurrent.length) {
+        System.arraycopy(arrCurrent, 0, arrValue, 0, arrCurrent.length);
+      } else {
+        if (arrValue.length != arrCurrent.length) {
+          throw new IOException(MessageFormat.format("HoodieMergeLogReader : 
size of object differs. Value size :  {0}, Current Object size : {1}",
+              arrValue.length, arrCurrent.length));
+        } else {
+          throw new IOException("HoodieMergeLogReader can not support 
RecordReaders that don't return same key & value & value is null");
+        }
+      }
+    }
+
+    offset++;
+    return true;
+  }
+
+  @Override
+  public NullWritable createKey() {
+    return null;
+  }
+
+  @Override
+  public ArrayWritable createValue() {
+    return valueObj;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return offset;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (logRecordScanner != null) {
+      logRecordScanner.close();
+    }
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    if (end == offset) {
+      return 0.0f;

Review comment:
       Is this correct ?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##########
@@ -336,6 +336,11 @@ public static String getFileExtensionFromLog(Path logPath) 
{
     return matcher.group(3);
   }
 
+  public static String getLogFileExtension(String fullName) {
+    Matcher matcher = LOG_FILE_PATTERN.matcher(fullName);

Review comment:
       What's the difference with `getFileExtensionFromLog` ?

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
##########
@@ -445,10 +457,21 @@ public static HoodieMetadataConfig 
buildMetadataConfig(Configuration conf) {
         HoodieTableFileSystemView fsView = 
fsViewCache.computeIfAbsent(metaClient, tableMetaClient ->
             
FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, 
tableMetaClient, buildMetadataConfig(job), timeline));
         List<HoodieBaseFile> filteredBaseFiles = new ArrayList<>();
+        Map<HoodieLogFile, List<String>> filteredLogs = new HashMap<>();
         for (Path p : entry.getValue()) {

Review comment:
       The cache key is weird.

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class BaseFileWithLogsSplit extends FileSplit {

Review comment:
       Add class document.

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java
##########
@@ -119,6 +275,11 @@ void addProjectionToJobConf(final RealtimeSplit 
realtimeSplit, final JobConf job
     addProjectionToJobConf(realtimeSplit, jobConf);
     LOG.info("Creating record reader with readCols :" + 
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)
         + ", Ids :" + 
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
+
+    // for log only split, we no need parquet reader, set it to empty
+    if (realtimeSplit.getPath().toString().contains(".log.")) {
+      return new HoodieRealtimeRecordReader(realtimeSplit, jobConf, new 
HoodieEmptyRecordReader());
+    }

Review comment:
       Use `FSUtils.isLogFile` ?

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.Path;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * we need to encode additional information in Path to track matching log file 
and base files.
+ * Hence, this weird looking class which tracks an log/base file status
+ */
+public class PathWithLogFilePath extends Path {
+
+  private boolean belongToIncrementalPath = false;
+  private List<String> deltaLogPaths = new ArrayList<>();

Review comment:
       Ad comments for each member.

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
##########
@@ -55,13 +55,20 @@
   private final Set<String> deltaRecordKeys;
   private int recordKeyIndex = 
HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS;
   private Iterator<String> deltaItr;
+  private boolean logFileOnlySplit;
+  private HoodieMergedLogReader logReader;
 
   public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job,
       RecordReader<NullWritable, ArrayWritable> realReader) throws IOException 
{
     super(split, job);
     this.parquetReader = realReader;
-    this.deltaRecordMap = getMergedLogRecordScanner().getRecords();
+    HoodieMergedLogRecordScanner hoodieMergedLogRecordScanner = 
getMergedLogRecordScanner();
+    this.deltaRecordMap = hoodieMergedLogRecordScanner.getRecords();
     this.deltaRecordKeys = new HashSet<>(this.deltaRecordMap.keySet());
+    if (split.getPath().toString().contains(".log.")) {
+      this.logFileOnlySplit = true;

Review comment:
       Keep `logReader` as `Option< HoodieMergedLogReader >` then we can 
eliminate the flag `logFileOnlySplit`.

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java
##########
@@ -66,6 +90,138 @@
     return HoodieRealtimeInputFormatUtils.getRealtimeSplits(job, fileSplits);
   }
 
+  /**
+   * keep the logical of mor_incr_view as same as spark datasource.
+   * to do: unify the incremental view code between hive/spark-sql and spark 
datasource

Review comment:
       Can we unify the logic in this PR ?

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
##########
@@ -55,13 +55,20 @@
   private final Set<String> deltaRecordKeys;
   private int recordKeyIndex = 
HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS;
   private Iterator<String> deltaItr;
+  private boolean logFileOnlySplit;
+  private HoodieMergedLogReader logReader;
 
   public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job,
       RecordReader<NullWritable, ArrayWritable> realReader) throws IOException 
{
     super(split, job);
     this.parquetReader = realReader;
-    this.deltaRecordMap = getMergedLogRecordScanner().getRecords();
+    HoodieMergedLogRecordScanner hoodieMergedLogRecordScanner = 
getMergedLogRecordScanner();
+    this.deltaRecordMap = hoodieMergedLogRecordScanner.getRecords();
     this.deltaRecordKeys = new HashSet<>(this.deltaRecordMap.keySet());
+    if (split.getPath().toString().contains(".log.")) {
+      this.logFileOnlySplit = true;

Review comment:
       Use `FSUtils.isLogFile` ?

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java
##########
@@ -120,28 +147,32 @@
             .orElse(Stream.empty());
 
         // subgroup splits again by file id & match with log files.
-        Map<String, List<FileSplit>> groupedInputSplits = 
partitionsToParquetSplits.get(partitionPath).stream()
-            .collect(Collectors.groupingBy(split -> 
FSUtils.getFileId(split.getPath().getName())));
+        Map<String, List<FileSplit>> groupedInputSplits = 
filteredPartitionsToParquetSplits.get(partitionPath).stream()
+            .collect(Collectors.groupingBy(split -> {
+              String fileName = split.getPath().getName();
+              // mor snapsht view may contains some raw log files, for 
example: if user use Hbase index, hudi will produce only log Files
+              if (split instanceof BaseFileWithLogsSplit && 
((BaseFileWithLogsSplit) split).getBaseFilePath().isEmpty()) {
+                fileName = split.getPath().getName().substring(1, 
fileName.length() - 1);
+              }

Review comment:
       `FSUtils.getFileIdFromLogPath` ?

##########
File path: 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java
##########
@@ -66,6 +90,138 @@
     return HoodieRealtimeInputFormatUtils.getRealtimeSplits(job, fileSplits);
   }
 
+  /**
+   * keep the logical of mor_incr_view as same as spark datasource.
+   * to do: unify the incremental view code between hive/spark-sql and spark 
datasource
+   */
+  @Override
+  protected List<FileStatus> listStatusForIncrementalMode(
+      JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> 
inputPaths) throws IOException {
+    List<FileStatus> result = new ArrayList<>();
+    String tableName = tableMetaClient.getTableConfig().getTableName();
+    Job jobContext = Job.getInstance(job);
+
+    Option<HoodieTimeline> timeline = 
HoodieInputFormatUtils.getFilteredCommitsTimeline(jobContext, tableMetaClient);
+    if (!timeline.isPresent()) {
+      return result;
+    }
+    String lastIncrementalTs = HoodieHiveUtils.readStartCommitTime(jobContext, 
tableName);
+    // Total number of commits to return in this batch. Set this to -1 to get 
all the commits.
+    Integer maxCommits = HoodieHiveUtils.readMaxCommits(jobContext, tableName);
+    HoodieTimeline commitsTimelineToReturn = 
timeline.get().findInstantsAfter(lastIncrementalTs, maxCommits);
+    Option<List<HoodieInstant>> commitsToCheck = 
Option.of(commitsTimelineToReturn.getInstants().collect(Collectors.toList()));
+    if (!commitsToCheck.isPresent()) {
+      return result;
+    }
+    Map<String, HashMap<String, FileStatus>> partitionsWithFileStatus  = 
HoodieInputFormatUtils
+        .listAffectedFilesForCommits(new Path(tableMetaClient.getBasePath()), 
commitsToCheck.get(), commitsTimelineToReturn);
+    // build fileGroup from fsView
+    List<FileStatus> affectedFileStatus = new ArrayList<>();
+    partitionsWithFileStatus.forEach((key, value) -> value.forEach((k, v) -> 
affectedFileStatus.add(v)));
+    HoodieTableFileSystemView fsView = new 
HoodieTableFileSystemView(tableMetaClient, commitsTimelineToReturn, 
affectedFileStatus.toArray(new FileStatus[0]));
+    // build fileGroup from fsView
+    String basePath = tableMetaClient.getBasePath();
+    // filter affectedPartition by inputPaths
+    List<String> affectedPartition = partitionsWithFileStatus.keySet().stream()
+        .filter(k -> k.isEmpty() ? inputPaths.contains(new Path(basePath)) : 
inputPaths.contains(new Path(basePath, k))).collect(Collectors.toList());
+    if (affectedPartition.isEmpty()) {
+      return result;
+    }
+    List<HoodieFileGroup> fileGroups = affectedPartition.stream()
+        .flatMap(partitionPath -> 
fsView.getAllFileGroups(partitionPath)).collect(Collectors.toList());
+    setInputPaths(job, affectedPartition.stream()
+        .map(p -> p.isEmpty() ? basePath : new Path(basePath, 
p).toUri().toString()).collect(Collectors.joining(",")));
+
+    // find all file status in current partitionPath
+    FileStatus[] fileStatuses = getStatus(job);
+    Map<String, FileStatus> candidateFileStatus = new HashMap<>();
+    for (int i = 0; i < fileStatuses.length; i++) {
+      String key = fileStatuses[i].getPath().toString();
+      candidateFileStatus.put(key, fileStatuses[i]);
+    }
+
+    String maxCommitTime = fsView.getLastInstant().get().getTimestamp();
+    fileGroups.stream().forEach(f -> {
+      try {
+        List<FileSlice> baseFiles = f.getAllFileSlices().filter(slice -> 
slice.getBaseFile().isPresent()).collect(Collectors.toList());
+        if (!baseFiles.isEmpty()) {
+          FileStatus baseFileStatus = 
HoodieInputFormatUtils.getFileStatus(baseFiles.get(0).getBaseFile().get());
+          String baseFilePath = baseFileStatus.getPath().toUri().toString();
+          if (!candidateFileStatus.containsKey(baseFilePath)) {
+            throw new HoodieException("Error obtaining fileStatus for file: " 
+ baseFilePath);
+          }
+          RealtimeFileStatus fileStatus = new 
RealtimeFileStatus(candidateFileStatus.get(baseFilePath));
+          fileStatus.setMaxCommitTime(maxCommitTime);
+          fileStatus.setBelongToIncrementalFileStatus(true);
+          fileStatus.setBasePath(basePath);
+          fileStatus.setBaseFilePath(baseFilePath);
+          
fileStatus.setDeltaLogPaths(f.getLatestFileSlice().get().getLogFiles().map(l -> 
l.getPath().toString()).collect(Collectors.toList()));
+          // try to set bootstrapfileStatus
+          if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile 
|| baseFileStatus instanceof FileStatusWithBootstrapBaseFile) {
+            fileStatus.setBootStrapFileStatus(baseFileStatus);
+          }
+          result.add(fileStatus);
+        }
+        // add file group which has only logs.
+        if (f.getLatestFileSlice().isPresent() && baseFiles.isEmpty()) {
+          List<FileStatus> logFileStatus = 
f.getLatestFileSlice().get().getLogFiles().map(logFile -> 
logFile.getFileStatus()).collect(Collectors.toList());
+          if (logFileStatus.size() > 0) {
+            RealtimeFileStatus fileStatus = new 
RealtimeFileStatus(logFileStatus.get(0));
+            fileStatus.setBelongToIncrementalFileStatus(true);
+            fileStatus.setDeltaLogPaths(logFileStatus.stream().map(l -> 
l.getPath().toString()).collect(Collectors.toList()));
+            fileStatus.setMaxCommitTime(maxCommitTime);
+            fileStatus.setBasePath(basePath);
+            result.add(fileStatus);
+          }
+        }
+      } catch (IOException e) {
+        throw new HoodieException("Error obtaining data file/log file grouping 
", e);
+      }
+    });
+    return result;
+  }
+
+  @Override
+  protected boolean includeLogFilesForSnapShotView() {
+    return true;
+  }
+
+  @Override
+  protected boolean isSplitable(FileSystem fs, Path filename) {
+    if (filename instanceof PathWithLogFilePath) {
+      return ((PathWithLogFilePath)filename).splitable();
+    }
+    return super.isSplitable(fs, filename);
+  }
+
+  @Override
+  protected FileSplit makeSplit(Path file, long start, long length, String[] 
hosts) {
+    if (file instanceof PathWithLogFilePath) {
+      PathWithLogFilePath pf = (PathWithLogFilePath) file;
+      if (pf.getPathWithBootstrapFileStatus() == null) {
+        return pf.buildSplit(file, start, length, hosts);
+      } else {

Review comment:
       add a method `doMakeSplit` to abstract the variable `String[] 
inMemoryHosts`, then we only need one piece of code.




-- 
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