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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala:
##########
@@ -304,9 +304,8 @@ object DefaultSource {
         Option(schema)
       }
 
-      val useNewParquetFileFormat = 
parameters.getOrElse(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(),
-        
HoodieReaderConfig.FILE_GROUP_READER_ENABLED.defaultValue().toString).toBoolean 
&&
-        !metaClient.isMetadataTable && (globPaths == null || globPaths.isEmpty)
+      val useNewParquetFileFormat = !metaClient.isMetadataTable && (globPaths 
== null || globPaths.isEmpty) &&
+        
HoodieReaderConfig.isFileGroupReaderEnabled(metaClient.getTableConfig.getTableVersion,
 parameters.asJava)

Review Comment:
   Similarly, remove this.



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java:
##########
@@ -527,19 +529,39 @@ public static String getTableBasePath(InputSplit split, 
JobConf jobConf) throws
       return realtimeSplit.getBasePath();
     } else {
       Path inputPath = ((FileSplit) split).getPath();
-      FileSystem fs = inputPath.getFileSystem(jobConf);
-      HoodieStorage storage = new HoodieHadoopStorage(fs);
-      Option<StoragePath> tablePath = TablePathUtils.getTablePath(storage, 
convertToStoragePath(inputPath));
-      return tablePath.get().toString();
+      return getTablePath(jobConf, inputPath);
     }
   }
 
+  private static String getTablePath(JobConf jobConf, Path inputPath) throws 
IOException {
+    FileSystem fs = inputPath.getFileSystem(jobConf);
+    HoodieStorage storage = new HoodieHadoopStorage(fs);
+    Option<StoragePath> tablePath = TablePathUtils.getTablePath(storage, 
convertToStoragePath(inputPath));
+    return tablePath.get().toString();
+  }
+
   /**
    * `schema.on.read` and skip merge not implemented
    */
-  public static boolean shouldUseFilegroupReader(final JobConf jobConf, final 
InputSplit split) {
-    return 
jobConf.getBoolean(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), 
HoodieReaderConfig.FILE_GROUP_READER_ENABLED.defaultValue())
-        && 
!jobConf.getBoolean(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key(), 
HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.defaultValue())
-        && !(split instanceof BootstrapBaseFileSplit);
+  public static boolean shouldUseFilegroupReader(final JobConf jobConf, final 
InputSplit split) throws IOException {
+    if (split instanceof FileSplit || split instanceof RealtimeSplit) {
+      HoodieTableMetaClient metaClient = 
HoodieTableMetaClient.builder().setConf(getStorageConf(jobConf)).setBasePath(getTableBasePath(split,
 jobConf)).build();
+      return 
HoodieReaderConfig.isFileGroupReaderEnabled(metaClient.getTableConfig().getTableVersion(),
 jobConf)
+          && 
!jobConf.getBoolean(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key(), 
HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.defaultValue())
+          && !(split instanceof BootstrapBaseFileSplit);
+    } else if (split instanceof CombineFileSplit) {
+      for (Path path : ((CombineFileSplit) split).getPaths()) {
+        HoodieTableMetaClient metaClient = 
HoodieTableMetaClient.builder().setConf(getStorageConf(jobConf)).setBasePath(getTablePath(jobConf,
 path)).build();
+        boolean isFileGroupReaderEnabled = 
HoodieReaderConfig.isFileGroupReaderEnabled(metaClient.getTableConfig().getTableVersion(),
 jobConf)

Review Comment:
   Similarly, let's remove this logic and fix the file group reader logic 
forward.



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