yihua commented on code in PR #11649:
URL: https://github.com/apache/hudi/pull/11649#discussion_r1690491303


##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java:
##########
@@ -149,14 +120,23 @@ public HoodieStorage getStorage(String path, 
StorageConfiguration<?> conf) {
 
   @Override
   public ClosableIterator<ArrayWritable> getFileRecordIterator(StoragePath 
filePath, long start, long length, Schema dataSchema, Schema requiredSchema, 
HoodieStorage storage) throws IOException {
-    JobConf jobConfCopy = new JobConf(jobConf);
+    JobConf jobConfCopy = new 
JobConf(storage.getConf().unwrapAs(Configuration.class));
+    if (getNeedsBootstrapMerge()) {
+      // Hive PPD works at row-group level and only enabled when 
hive.optimize.index.filter=true;
+      // The above config is disabled by default. But when enabled, would 
cause misalignment between
+      // skeleton and bootstrap file. We will disable them specifically when 
query needs bootstrap and skeleton
+      // file to be stitched.
+      // This disables row-group filtering
+      jobConfCopy.unset(TableScanDesc.FILTER_EXPR_CONF_STR);
+      jobConfCopy.unset(ConvertAstToSearchArg.SARG_PUSHDOWN);
+    }
     //move the partition cols to the end, because in some cases it has issues 
if we don't do that
     Schema modifiedDataSchema = 
HoodieAvroUtils.generateProjectionSchema(dataSchema, 
Stream.concat(dataSchema.getFields().stream()
             .map(f -> f.name().toLowerCase(Locale.ROOT)).filter(n -> 
!partitionColSet.contains(n)),
         partitionCols.stream().filter(c -> dataSchema.getField(c) != 
null)).collect(Collectors.toList()));
     setSchemas(jobConfCopy, modifiedDataSchema, requiredSchema);
-    InputSplit inputSplit = new FileSplit(new Path(filePath.toString()), 
start, length, hosts.get(filePath.toString()));
-    RecordReader<NullWritable, ArrayWritable> recordReader = 
readerCreator.getRecordReader(inputSplit, jobConfCopy, reporter);
+    InputSplit inputSplit = new FileSplit(new Path(filePath.toString()), 
start, length, (String[]) null);

Review Comment:
   Does this affect performance as the location/hosts are not provided?



##########
hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.HoodieJavaWriteClient;
+import org.apache.hudi.client.common.HoodieJavaEngineContext;
+import org.apache.hudi.common.config.HoodieMemoryConfig;
+import org.apache.hudi.common.config.HoodieReaderConfig;
+import org.apache.hudi.common.config.RecordMergeMode;
+import org.apache.hudi.common.engine.EngineType;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordMerger;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.read.CustomPayloadForTesting;
+import org.apache.hudi.common.table.read.TestHoodieFileGroupReaderBase;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat;
+import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
+import org.apache.hudi.hadoop.utils.HoodieArrayWritableAvroUtils;
+import org.apache.hudi.storage.HoodieStorage;
+import org.apache.hudi.storage.StorageConfiguration;
+import org.apache.hudi.storage.hadoop.HoodieHadoopStorage;
+import org.apache.hudi.testutils.HoodieJavaClientTestHarness;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.mr.ExecMapper;
+import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+
+import static org.apache.hadoop.hive.ql.exec.Utilities.HAS_MAP_WORK;
+import static org.apache.hadoop.hive.ql.exec.Utilities.MAPRED_MAPPER_CLASS;
+import static 
org.apache.hudi.hadoop.HoodieFileGroupReaderBasedRecordReader.getRecordKeyField;
+import static 
org.apache.hudi.hadoop.HoodieFileGroupReaderBasedRecordReader.getStoredPartitionFieldNames;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+public class TestHoodieFileGroupReaderOnHive extends 
TestHoodieFileGroupReaderBase<ArrayWritable> {
+
+  @Override
+  @Disabled
+  public void testReadLogFilesOnlyInMergeOnReadTable(RecordMergeMode 
recordMergeMode, String logDataBlockFormat) throws Exception {}
+
+  private static final String PARTITION_COLUMN = "datestr";
+  private static JobConf baseJobConf;
+  private static HdfsTestService hdfsTestService;
+  private static HoodieStorage storage;
+  private static FileSystem fs;
+  private static StorageConfiguration<Configuration> storageConf;
+
+  @BeforeAll
+  public static void setUpClass() throws IOException, InterruptedException {
+    // Append is not supported in LocalFileSystem. HDFS needs to be setup.
+    hdfsTestService = new HdfsTestService();
+    fs = hdfsTestService.start(true).getFileSystem();
+    storageConf = HoodieTestUtils.getDefaultStorageConf();
+    baseJobConf = new JobConf(storageConf.unwrap());
+    baseJobConf.set(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.key(), 
String.valueOf(1024 * 1024));
+    fs.setConf(baseJobConf);
+    storage = new HoodieHadoopStorage(fs);
+  }
+
+  @AfterAll
+  public static void tearDownClass() throws IOException {
+    hdfsTestService.stop();
+    if (fs != null) {
+      fs.close();
+      storage.close();
+    }
+  }
+
+  @Override
+  public StorageConfiguration<?> getStorageConf() {
+    return storageConf;
+  }
+
+  @Override
+  public String getBasePath() {
+    return tempDir.toAbsolutePath() + "/myTable";
+  }
+
+  @Override
+  public HoodieReaderContext<ArrayWritable> getHoodieReaderContext(String 
tablePath, Schema avroSchema, StorageConfiguration<?> storageConf) {
+    HoodieFileGroupReaderBasedRecordReader.HiveReaderCreator readerCreator = 
(s, j) -> new MapredParquetInputFormat().getRecordReader(s, j, null);
+    HoodieTableMetaClient metaClient = 
HoodieTableMetaClient.builder().setConf(storageConf).setBasePath(tablePath).build();
+    String tableName = metaClient.getTableConfig().getTableName();
+    JobConf jobConf = new JobConf(storageConf.unwrapAs(Configuration.class));
+    setupJobconf(jobConf);
+    HoodieArrayWritableAvroUtils.initCacheForTable(tableName, avroSchema, 
jobConf);
+    return new HiveHoodieReaderContext(readerCreator, tableName, 
getRecordKeyField(metaClient),
+        getStoredPartitionFieldNames(new 
JobConf(storageConf.unwrapAs(Configuration.class)), avroSchema));
+
+  }
+
+  @Override
+  public String getRecordPayloadForMergeMode(RecordMergeMode mergeMode) {
+    switch (mergeMode) {
+      case EVENT_TIME_ORDERING:
+        return DefaultHoodieRecordPayload.class.getName();
+      case OVERWRITE_WITH_LATEST:
+        return OverwriteWithLatestAvroPayload.class.getName();
+      case CUSTOM:
+      default:
+        return CustomPayloadForTesting.class.getName();
+    }
+  }
+
+  @Override
+  public void commitToTable(List<HoodieRecord> recordList, String operation, 
Map<String, String> writeConfigs) {
+    HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
+        .withEngineType(EngineType.JAVA)
+        .withEmbeddedTimelineServerEnabled(false)
+        .withProps(writeConfigs)
+        .withPath(getBasePath())
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
+        .build();
+
+    HoodieJavaClientTestHarness.TestJavaTaskContextSupplier 
taskContextSupplier = new 
HoodieJavaClientTestHarness.TestJavaTaskContextSupplier();
+    HoodieJavaEngineContext context = new 
HoodieJavaEngineContext(getStorageConf(), taskContextSupplier);
+    //init table if not exists
+    try {
+      try (RawLocalFileSystem lfs = new RawLocalFileSystem()) {
+        lfs.setConf(baseJobConf);
+        boolean basepathExists = lfs.exists(new Path(getBasePath()));
+        boolean operationIsInsert = operation.equalsIgnoreCase("insert");
+        if (!basepathExists || operationIsInsert) {
+          if (basepathExists) {
+            lfs.delete(new Path(getBasePath()), true);
+          }
+          String recordMergerStrategy = "";
+          if 
(RecordMergeMode.valueOf(writeConfigs.get("hoodie.record.merge.mode")).equals(RecordMergeMode.OVERWRITE_WITH_LATEST))
 {
+            recordMergerStrategy = 
HoodieRecordMerger.OVERWRITE_MERGER_STRATEGY_UUID;
+          } else if 
(RecordMergeMode.valueOf(writeConfigs.get("hoodie.record.merge.mode")).equals(RecordMergeMode.EVENT_TIME_ORDERING))
 {
+            recordMergerStrategy = 
HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID;
+          } else if 
(RecordMergeMode.valueOf(writeConfigs.get("hoodie.record.merge.mode")).equals(RecordMergeMode.CUSTOM))
 {
+            //match the behavior of spark for now, but this should be a config
+            recordMergerStrategy = 
HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID;
+          }
+          Map<String, Object> initConfigs = new HashMap<>(writeConfigs);
+          HoodieTableMetaClient.withPropertyBuilder()
+              
.setTableType(writeConfigs.getOrDefault("hoodie.datasource.write.table.type", 
"MERGE_ON_READ"))
+              .setTableName(writeConfigs.get("hoodie.table.name"))
+              
.setPartitionFields(writeConfigs.getOrDefault("hoodie.datasource.write.partitionpath.field",
 ""))
+              
.setRecordMergeMode(RecordMergeMode.valueOf(writeConfigs.get("hoodie.record.merge.mode")))
+              .setRecordMergerStrategy(recordMergerStrategy)
+              .set(initConfigs).initTable(storageConf, getBasePath());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+
+    HoodieJavaWriteClient writeClient = new HoodieJavaWriteClient(context, 
writeConfig);
+    String instantTime = writeClient.createNewInstantTime();
+    writeClient.startCommitWithTime(instantTime);
+    if (operation.toLowerCase().equals("insert")) {
+      writeClient.insert(recordList, instantTime);
+    } else {
+      writeClient.upsert(recordList, instantTime);
+    }
+  }
+
+  @Override
+  public void validateRecordsInFileGroup(String tablePath, List<ArrayWritable> 
actualRecordList, Schema schema, String fileGroupId) {
+    
assertEquals(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_SCHEMA),
 schema);
+    try {
+      //prepare fg reader records to be compared to the baseline reader
+      HoodieReaderContext<ArrayWritable> readerContext = 
getHoodieReaderContext(tablePath, schema, storageConf);
+      Map<String, ArrayWritable> recordMap = new HashMap<>();
+      for (ArrayWritable record : actualRecordList) {
+        recordMap.put(readerContext.getRecordKey(record, schema), record);
+      }
+
+
+      JobConf jobConf = new JobConf(baseJobConf);
+      jobConf.set(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false");
+
+      TableDesc tblDesc = Utilities.defaultTd;
+      // Set the input format

Review Comment:
   Could you wrap some part of the setup and config prep into separate 
method(s) for readability?



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java:
##########
@@ -101,8 +102,7 @@ org.apache.hadoop.mapred.RecordReader<NullWritable, 
ArrayWritable> getRecordRead
 
   public HoodieFileGroupReaderBasedRecordReader(HiveReaderCreator 
readerCreator,
                                                 final InputSplit split,
-                                                final JobConf jobConf,
-                                                final Reporter reporter) 
throws IOException {

Review Comment:
   What is the reporter used for?



##########
hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.HoodieJavaWriteClient;
+import org.apache.hudi.client.common.HoodieJavaEngineContext;
+import org.apache.hudi.common.config.HoodieMemoryConfig;
+import org.apache.hudi.common.config.HoodieReaderConfig;
+import org.apache.hudi.common.config.RecordMergeMode;
+import org.apache.hudi.common.engine.EngineType;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordMerger;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.read.CustomPayloadForTesting;
+import org.apache.hudi.common.table.read.TestHoodieFileGroupReaderBase;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat;
+import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
+import org.apache.hudi.hadoop.utils.HoodieArrayWritableAvroUtils;
+import org.apache.hudi.storage.HoodieStorage;
+import org.apache.hudi.storage.StorageConfiguration;
+import org.apache.hudi.storage.hadoop.HoodieHadoopStorage;
+import org.apache.hudi.testutils.HoodieJavaClientTestHarness;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.mr.ExecMapper;
+import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+
+import static org.apache.hadoop.hive.ql.exec.Utilities.HAS_MAP_WORK;
+import static org.apache.hadoop.hive.ql.exec.Utilities.MAPRED_MAPPER_CLASS;
+import static 
org.apache.hudi.hadoop.HoodieFileGroupReaderBasedRecordReader.getRecordKeyField;
+import static 
org.apache.hudi.hadoop.HoodieFileGroupReaderBasedRecordReader.getStoredPartitionFieldNames;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+public class TestHoodieFileGroupReaderOnHive extends 
TestHoodieFileGroupReaderBase<ArrayWritable> {
+
+  @Override
+  @Disabled
+  public void testReadLogFilesOnlyInMergeOnReadTable(RecordMergeMode 
recordMergeMode, String logDataBlockFormat) throws Exception {}
+
+  private static final String PARTITION_COLUMN = "datestr";
+  private static JobConf baseJobConf;
+  private static HdfsTestService hdfsTestService;
+  private static HoodieStorage storage;
+  private static FileSystem fs;
+  private static StorageConfiguration<Configuration> storageConf;
+
+  @BeforeAll
+  public static void setUpClass() throws IOException, InterruptedException {
+    // Append is not supported in LocalFileSystem. HDFS needs to be setup.
+    hdfsTestService = new HdfsTestService();
+    fs = hdfsTestService.start(true).getFileSystem();
+    storageConf = HoodieTestUtils.getDefaultStorageConf();
+    baseJobConf = new JobConf(storageConf.unwrap());
+    baseJobConf.set(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.key(), 
String.valueOf(1024 * 1024));
+    fs.setConf(baseJobConf);
+    storage = new HoodieHadoopStorage(fs);
+  }
+
+  @AfterAll
+  public static void tearDownClass() throws IOException {
+    hdfsTestService.stop();
+    if (fs != null) {
+      fs.close();
+      storage.close();
+    }
+  }
+
+  @Override
+  public StorageConfiguration<?> getStorageConf() {
+    return storageConf;
+  }
+
+  @Override
+  public String getBasePath() {
+    return tempDir.toAbsolutePath() + "/myTable";
+  }
+
+  @Override
+  public HoodieReaderContext<ArrayWritable> getHoodieReaderContext(String 
tablePath, Schema avroSchema, StorageConfiguration<?> storageConf) {
+    HoodieFileGroupReaderBasedRecordReader.HiveReaderCreator readerCreator = 
(s, j) -> new MapredParquetInputFormat().getRecordReader(s, j, null);
+    HoodieTableMetaClient metaClient = 
HoodieTableMetaClient.builder().setConf(storageConf).setBasePath(tablePath).build();
+    String tableName = metaClient.getTableConfig().getTableName();
+    JobConf jobConf = new JobConf(storageConf.unwrapAs(Configuration.class));
+    setupJobconf(jobConf);
+    HoodieArrayWritableAvroUtils.initCacheForTable(tableName, avroSchema, 
jobConf);
+    return new HiveHoodieReaderContext(readerCreator, tableName, 
getRecordKeyField(metaClient),
+        getStoredPartitionFieldNames(new 
JobConf(storageConf.unwrapAs(Configuration.class)), avroSchema));
+
+  }
+
+  @Override
+  public String getRecordPayloadForMergeMode(RecordMergeMode mergeMode) {
+    switch (mergeMode) {
+      case EVENT_TIME_ORDERING:
+        return DefaultHoodieRecordPayload.class.getName();
+      case OVERWRITE_WITH_LATEST:
+        return OverwriteWithLatestAvroPayload.class.getName();
+      case CUSTOM:
+      default:
+        return CustomPayloadForTesting.class.getName();
+    }
+  }
+
+  @Override
+  public void commitToTable(List<HoodieRecord> recordList, String operation, 
Map<String, String> writeConfigs) {
+    HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
+        .withEngineType(EngineType.JAVA)
+        .withEmbeddedTimelineServerEnabled(false)
+        .withProps(writeConfigs)
+        .withPath(getBasePath())
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
+        .build();
+
+    HoodieJavaClientTestHarness.TestJavaTaskContextSupplier 
taskContextSupplier = new 
HoodieJavaClientTestHarness.TestJavaTaskContextSupplier();
+    HoodieJavaEngineContext context = new 
HoodieJavaEngineContext(getStorageConf(), taskContextSupplier);
+    //init table if not exists
+    try {
+      try (RawLocalFileSystem lfs = new RawLocalFileSystem()) {

Review Comment:
   Do we need `RawLocalFileSystem`?  Could we derive the FS instance based on 
the table base path?



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/OverwriteWithLatestHiveRecordMerger.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.avro.Schema;
+
+import java.io.IOException;
+
+public class OverwriteWithLatestHiveRecordMerger extends 
HoodieHiveRecordMerger {

Review Comment:
   Given we're introducing the new record merger classes in the next release, 
it's OK to re-organize it for better understanding.  For Spark, I think it's OK 
to change the class structure and possibly rename the classes, as we don't 
expect users to explicitly set the merger class, unless it's custom.



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