vinothchandar commented on code in PR #12390:
URL: https://github.com/apache/hudi/pull/12390#discussion_r1865568298


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java:
##########
@@ -43,6 +45,11 @@
 public class HoodieFlinkMergeOnReadTableCompactor<T>
     extends HoodieCompactor<T, List<HoodieRecord<T>>, List<HoodieKey>, 
List<WriteStatus>> {
 
+  @Override
+  public Option<CompactorBroadcastManager> 
getCompactorBroadcastManager(HoodieEngineContext context) {

Review Comment:
   do this once in the base class? so you dont have to override everywhere?. 
anyways. minor comment



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -130,8 +134,24 @@ public HoodieData<WriteStatus> compact(
     TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier();
     // if this is a MDT, set up the instant range of log reader just like 
regular MDT snapshot reader.
     Option<InstantRange> instantRange = 
CompactHelpers.getInstance().getInstantRange(metaClient);
+
+    boolean useFileGroupReaderBasedCompaction = !metaClient.isMetadataTable()

Review Comment:
   today, we dont have partial updates in MT.. but is there any real reason 
this cannot work on MT?



##########
hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java:
##########
@@ -317,4 +317,9 @@ public Iterator<List<WriteStatus>> handleInsert(
     createHandle.write();
     return Collections.singletonList(createHandle.close()).iterator();
   }
+
+  @Override
+  public boolean supportsFileGroupReader() {

Review Comment:
   same. override to false at base class.. and then just make it true for spark 
alone



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import 
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ *     Check if there is a new record coming in. If yes, merge two records and 
write to file
+ *     else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ *     rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ *     rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to 
storage.
+ *    => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ *    => rec2_1 is written as is
+ *    => rec3_1 is written as is
+ *    => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ *    => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T, 
I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+  protected Set<String> writtenRecordKeys;
+  protected HoodieReaderContext readerContext;
+  protected HoodieFileWriter fileWriter;
+  protected boolean preserveMetadata = false;
+
+  protected StoragePath newFilePath;
+  protected StoragePath oldFilePath;
+  protected long recordsWritten = 0;
+  // TODO(yihua): audit delete stats because file group reader may not return 
deletes
+  protected long recordsDeleted = 0;
+  protected long updatedRecordsWritten = 0;
+  protected long insertRecordsWritten = 0;
+  protected Option<BaseKeyGenerator> keyGeneratorOpt;
+  protected FileSlice fileSlice;
+  private HoodieBaseFile baseFileToMerge;
+
+  protected Option<String[]> partitionFields = Option.empty();
+  protected Object[] partitionValues = new Object[0];
+  protected Configuration conf;
+
+  /**
+   * Called by compactor code path using the file group reader.
+   */
+  public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String 
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                                  CompactionOperation operation, 
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator> 
keyGeneratorOpt,
+                                  HoodieReaderContext readerContext, 
Configuration conf) {
+    super(config, instantTime, operation.getPartitionPath(), 
operation.getFileId(), hoodieTable, taskContextSupplier);
+    this.readerContext = readerContext;
+    this.conf = conf;
+    Option<HoodieBaseFile> baseFileOpt =
+        operation.getBaseFile(config.getBasePath(), 
operation.getPartitionPath());
+    List<HoodieLogFile> logFiles = 
operation.getDeltaFileNames().stream().map(p ->
+            new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
+                config.getBasePath(), operation.getPartitionPath()), p)))
+        .collect(Collectors.toList());
+    this.fileSlice = new FileSlice(
+        operation.getFileGroupId(),
+        operation.getBaseInstantTime(),
+        baseFileOpt.isPresent() ? baseFileOpt.get() : null,
+        logFiles);
+    ValidationUtils.checkArgument(baseFileOpt.isPresent(),
+        "Only supporting compaction with base file using file group reader in 
HoodieMergeHandleV2");
+    this.preserveMetadata = true;
+    init(fileId, this.partitionPath, baseFileOpt.get());
+    validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
+  }
+
+  private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator> 
keyGeneratorOpt, boolean populateMetaFields) {
+    ValidationUtils.checkArgument(populateMetaFields == 
!keyGeneratorOpt.isPresent());
+    this.keyGeneratorOpt = keyGeneratorOpt;
+  }
+
+  /**
+   * Extract old file path, initialize StorageWriter and WriteStatus.
+   */
+  private void init(String fileId, String partitionPath, HoodieBaseFile 
baseFileToMerge) {
+    LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + 
fileId);
+    this.baseFileToMerge = baseFileToMerge;
+    this.writtenRecordKeys = new HashSet<>();
+    writeStatus.setStat(new HoodieWriteStat());
+    try {
+      String latestValidFilePath = baseFileToMerge.getFileName();
+      writeStatus.getStat().setPrevCommit(baseFileToMerge.getCommitTime());
+      // At the moment, we only support SI for overwrite with latest payload. 
So, we don't need to embed entire file slice here.
+      // HUDI-8518 will be taken up to fix it for any payload during which we 
might require entire file slice to be set here.
+      // Already AppendHandle adds all logs file from current file slice to 
HoodieDeltaWriteStat.
+      writeStatus.getStat().setPrevBaseFile(latestValidFilePath);
+
+      HoodiePartitionMetadata partitionMetadata = new 
HoodiePartitionMetadata(storage, instantTime,
+          new StoragePath(config.getBasePath()),
+          FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+          hoodieTable.getPartitionMetafileFormat());
+      partitionMetadata.trySave();
+
+      String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken, 
fileId, hoodieTable.getBaseFileExtension());
+      makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
+
+      LOG.info(String.format("Merging new data into oldPath %s, as newPath 
%s", oldFilePath.toString(),
+          newFilePath.toString()));
+      // file name is same for all records, in this bunch
+      writeStatus.setFileId(fileId);
+      writeStatus.setPartitionPath(partitionPath);
+      writeStatus.getStat().setPartitionPath(partitionPath);
+      writeStatus.getStat().setFileId(fileId);
+      setWriteStatusPath();
+
+      // Create Marker file,
+      // uses name of `newFilePath` instead of `newFileName`
+      // in case the sub-class may roll over the file handle name.
+      createMarkerFile(partitionPath, newFilePath.getName());
+
+      // Create the writer for writing the new version file
+      fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, 
newFilePath, hoodieTable.getStorage(),
+          config, writeSchemaWithMetaFields, taskContextSupplier, 
HoodieRecord.HoodieRecordType.SPARK);
+    } catch (IOException io) {
+      LOG.error("Error in update task at commit " + instantTime, io);
+      writeStatus.setGlobalError(io);
+      throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle 
for FileId: " + fileId + " on commit "
+          + instantTime + " on path " + 
hoodieTable.getMetaClient().getBasePath(), io);
+    }
+  }
+
+  protected void setWriteStatusPath() {
+    writeStatus.getStat().setPath(new StoragePath(config.getBasePath()), 
newFilePath);
+  }
+
+  protected void makeOldAndNewFilePaths(String partitionPath, String 
oldFileName, String newFileName) {
+    oldFilePath = makeNewFilePath(partitionPath, oldFileName);
+    newFilePath = makeNewFilePath(partitionPath, newFileName);
+  }
+
+  // TODO(yihua): is this still needed
+  /*
+  if (needsUpdateLocation()) {
+        record.unseal();
+        record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
+        record.seal();
+      }
+   */
+
+  public void write() {
+    boolean usePosition = 
config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS);
+    Schema readerSchema;
+    Option<InternalSchema> internalSchemaOption = Option.empty();
+    if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
+      readerSchema = HoodieAvroUtils.addMetadataFields(
+          new Schema.Parser().parse(config.getSchema()), 
config.allowOperationMetadataField());
+      internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
+    } else {
+      readerSchema = HoodieAvroUtils.addMetadataFields(
+          new Schema.Parser().parse(config.getSchema()), 
config.allowOperationMetadataField());
+    }
+    // TODO(yihua): reader schema is good enough for writer?

Review Comment:
   what do we use on the existing merge handle, lets be sure to match that. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -161,66 +162,97 @@ public List<WriteStatus> compact(HoodieCompactionHandler 
compactionHandler,
                                    Option<InstantRange> instantRange,
                                    TaskContextSupplier taskContextSupplier,
                                    CompactionExecutionHelper executionHelper) 
throws IOException {
-    HoodieStorage storage = metaClient.getStorage();
-    Schema readerSchema;
-    Option<InternalSchema> internalSchemaOption = Option.empty();
-    if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
-      readerSchema = new Schema.Parser().parse(config.getSchema());
-      internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
-      // its safe to modify config here, since we are running in task side.
-      ((HoodieTable) compactionHandler).getConfig().setDefault(config);
+    if 
(config.getBooleanOrDefault(HoodieReaderConfig.FILE_GROUP_READER_ENABLED)
+        && compactionHandler.supportsFileGroupReader()) {
+      List<WriteStatus> writeStatusList = 
compactionHandler.runCompactionUsingFileGroupReader(instantTime,
+          operation.getPartitionPath(), operation.getFileId(), operation, 2);
+      writeStatusList
+          .forEach(s -> {
+            final HoodieWriteStat stat = s.getStat();
+            /*

Review Comment:
   IIUC stats are set inside merge handle. So lets just remove all this?



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java:
##########
@@ -449,4 +455,9 @@ public Iterator<List<WriteStatus>> handleInsert(
     createHandle.write();
     return Collections.singletonList(createHandle.close()).iterator();
   }
+
+  @Override
+  public boolean supportsFileGroupReader() {

Review Comment:
   ok, if we are aligned on making FG reader the way forward for all engines 
(so that check will be eliminated at some point). Else method is more 
maintainable. lets still not let this block the PR. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -161,66 +162,97 @@ public List<WriteStatus> compact(HoodieCompactionHandler 
compactionHandler,
                                    Option<InstantRange> instantRange,
                                    TaskContextSupplier taskContextSupplier,
                                    CompactionExecutionHelper executionHelper) 
throws IOException {
-    HoodieStorage storage = metaClient.getStorage();
-    Schema readerSchema;
-    Option<InternalSchema> internalSchemaOption = Option.empty();
-    if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
-      readerSchema = new Schema.Parser().parse(config.getSchema());
-      internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
-      // its safe to modify config here, since we are running in task side.
-      ((HoodieTable) compactionHandler).getConfig().setDefault(config);
+    if 
(config.getBooleanOrDefault(HoodieReaderConfig.FILE_GROUP_READER_ENABLED)
+        && compactionHandler.supportsFileGroupReader()) {
+      List<WriteStatus> writeStatusList = 
compactionHandler.runCompactionUsingFileGroupReader(instantTime,
+          operation.getPartitionPath(), operation.getFileId(), operation, 2);
+      writeStatusList
+          .forEach(s -> {
+            final HoodieWriteStat stat = s.getStat();
+            /*
+            fill in log reading stats
+            
stat.setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
+            stat.setTotalLogFilesCompacted(scanner.getTotalLogFiles());
+            stat.setTotalLogRecords(scanner.getTotalLogRecords());
+            stat.setPartitionPath(operation.getPartitionPath());
+            stat
+                
.setTotalLogSizeCompacted(operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
+            stat.setTotalLogBlocks(scanner.getTotalLogBlocks());
+            stat.setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
+            stat.setTotalRollbackBlocks(scanner.getTotalRollbacks());
+            RuntimeStats runtimeStats = new RuntimeStats();
+            // scan time has to be obtained from scanner.
+            
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
+            // create and upsert time are obtained from the create or merge 
handle.
+            if (stat.getRuntimeStats() != null) {
+              
runtimeStats.setTotalCreateTime(stat.getRuntimeStats().getTotalCreateTime());
+              
runtimeStats.setTotalUpsertTime(stat.getRuntimeStats().getTotalUpsertTime());
+            }
+            stat.setRuntimeStats(runtimeStats);
+             */
+          });
+      return writeStatusList;
     } else {
-      readerSchema = HoodieAvroUtils.addMetadataFields(
-          new Schema.Parser().parse(config.getSchema()), 
config.allowOperationMetadataField());
-    }
-    LOG.info("Compaction operation started for base file: " + 
operation.getDataFileName() + " and delta files: " + 
operation.getDeltaFileNames()
-        + " for commit " + instantTime);
-    // TODO - FIX THIS
-    // Reads the entire avro file. Always only specific blocks should be read 
from the avro file
-    // (failure recover).
-    // Load all the delta commits since the last compaction commit and get all 
the blocks to be
-    // loaded and load it using CompositeAvroLogReader
-    // Since a DeltaCommit is not defined yet, reading all the records. 
revisit this soon.
+      HoodieStorage storage = metaClient.getStorage();

Review Comment:
   confirmed.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -160,67 +180,100 @@ public List<WriteStatus> compact(HoodieCompactionHandler 
compactionHandler,
                                    String maxInstantTime,
                                    Option<InstantRange> instantRange,
                                    TaskContextSupplier taskContextSupplier,
-                                   CompactionExecutionHelper executionHelper) 
throws IOException {
-    HoodieStorage storage = metaClient.getStorage();
-    Schema readerSchema;
-    Option<InternalSchema> internalSchemaOption = Option.empty();
-    if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
-      readerSchema = new Schema.Parser().parse(config.getSchema());
-      internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
-      // its safe to modify config here, since we are running in task side.
-      ((HoodieTable) compactionHandler).getConfig().setDefault(config);
+                                   CompactionExecutionHelper executionHelper,
+                                   boolean useFileGroupReaderBasedCompaction,
+                                   Option<CompactorBroadcastManager> 
broadcastManagerOpt) throws IOException {
+    if (useFileGroupReaderBasedCompaction) {

Review Comment:
   lets make two methods for if and else blocks for large blocks like this. 
makes the review easier (indentation stays same) and also more maintainable



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java:
##########
@@ -18,22 +18,53 @@
 
 package org.apache.hudi.client;
 
+import org.apache.hudi.SparkAdapterSupport$;
+import org.apache.hudi.SparkFileFormatInternalRowReaderContext;
 import org.apache.hudi.common.engine.EngineProperty;
+import org.apache.hudi.common.engine.HoodieReaderContext;
 import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieException;
 
 import org.apache.spark.SparkEnv;
 import org.apache.spark.TaskContext;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.execution.datasources.FileFormat;
+import org.apache.spark.sql.execution.datasources.parquet.SparkParquetReader;
+import org.apache.spark.sql.hudi.SparkAdapter;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.sources.Filter;
 import org.apache.spark.util.Utils;
 
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.function.Supplier;
 
+import scala.Tuple2;

Review Comment:
   do we need to depends on scala classes for sure.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java:
##########
@@ -18,22 +18,53 @@
 
 package org.apache.hudi.client;
 
+import org.apache.hudi.SparkAdapterSupport$;
+import org.apache.hudi.SparkFileFormatInternalRowReaderContext;
 import org.apache.hudi.common.engine.EngineProperty;
+import org.apache.hudi.common.engine.HoodieReaderContext;
 import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieException;
 
 import org.apache.spark.SparkEnv;
 import org.apache.spark.TaskContext;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.execution.datasources.FileFormat;
+import org.apache.spark.sql.execution.datasources.parquet.SparkParquetReader;
+import org.apache.spark.sql.hudi.SparkAdapter;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.sources.Filter;
 import org.apache.spark.util.Utils;
 
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.function.Supplier;
 
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
 /**
  * Spark task context supplier.
  */
 public class SparkTaskContextSupplier extends TaskContextSupplier implements 
Serializable {
+  private Option<SparkParquetReader> parquetReaderOpt = Option.empty();

Review Comment:
   this is a big no-no.. cannot be aware of parquet etc at this layer. 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java:
##########
@@ -90,4 +121,18 @@ public Option<String> getProperty(EngineProperty prop) {
     throw new HoodieException("Unknown engine property :" + prop);
   }
 
+  @Override
+  public Option<HoodieReaderContext> getReaderContext(HoodieTableMetaClient 
metaClient, boolean useReaderContext) {

Review Comment:
   this is a engine level class.. and this method feels  out of place.



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