This is an automated email from the ASF dual-hosted git repository.

sivabalan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 1ddf9d4f9d [HUDI-5076] Fixing non serializable path used in 
engineContext with metadata table intialization (#7036)
1ddf9d4f9d is described below

commit 1ddf9d4f9deeac81d78944a8d380dbd05d744a1e
Author: Sivabalan Narayanan <[email protected]>
AuthorDate: Wed Nov 2 17:10:07 2022 -0700

    [HUDI-5076] Fixing non serializable path used in engineContext with 
metadata table intialization (#7036)
---
 .../hudi/metadata/HoodieBackedTableMetadataWriter.java  | 17 +++++++++++------
 1 file changed, 11 insertions(+), 6 deletions(-)

diff --git 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
index 15759a570f..979a8d135a 100644
--- 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
+++ 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
@@ -68,6 +68,8 @@ import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIndexException;
 import org.apache.hudi.exception.HoodieMetadataException;
+import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.SerializablePath;
 
 import org.apache.avro.specific.SpecificRecordBase;
 import org.apache.hadoop.conf.Configuration;
@@ -616,23 +618,24 @@ public abstract class HoodieBackedTableMetadataWriter 
implements HoodieTableMeta
    * @return Map of partition names to a list of FileStatus for all the files 
in the partition
    */
   private List<DirectoryInfo> listAllPartitions(HoodieTableMetaClient 
datasetMetaClient) {
-    List<Path> pathsToList = new LinkedList<>();
-    pathsToList.add(new Path(dataWriteConfig.getBasePath()));
+    List<SerializablePath> pathsToList = new LinkedList<>();
+    pathsToList.add(new SerializablePath(new 
CachingPath(dataWriteConfig.getBasePath())));
 
     List<DirectoryInfo> partitionsToBootstrap = new LinkedList<>();
     final int fileListingParallelism = 
metadataWriteConfig.getFileListingParallelism();
     SerializableConfiguration conf = new 
SerializableConfiguration(datasetMetaClient.getHadoopConf());
     final String dirFilterRegex = 
dataWriteConfig.getMetadataConfig().getDirectoryFilterRegex();
     final String datasetBasePath = datasetMetaClient.getBasePath();
+    SerializablePath serializableBasePath = new SerializablePath(new 
CachingPath(datasetBasePath));
 
     while (!pathsToList.isEmpty()) {
       // In each round we will list a section of directories
       int numDirsToList = Math.min(fileListingParallelism, pathsToList.size());
       // List all directories in parallel
       List<DirectoryInfo> processedDirectories = 
engineContext.map(pathsToList.subList(0, numDirsToList), path -> {
-        FileSystem fs = path.getFileSystem(conf.get());
-        String relativeDirPath = FSUtils.getRelativePartitionPath(new 
Path(datasetBasePath), path);
-        return new DirectoryInfo(relativeDirPath, fs.listStatus(path));
+        FileSystem fs = path.get().getFileSystem(conf.get());
+        String relativeDirPath = 
FSUtils.getRelativePartitionPath(serializableBasePath.get(), path.get());
+        return new DirectoryInfo(relativeDirPath, fs.listStatus(path.get()));
       }, numDirsToList);
 
       pathsToList = new LinkedList<>(pathsToList.subList(numDirsToList, 
pathsToList.size()));
@@ -656,7 +659,9 @@ public abstract class HoodieBackedTableMetadataWriter 
implements HoodieTableMeta
           partitionsToBootstrap.add(dirInfo);
         } else {
           // Add sub-dirs to the queue
-          pathsToList.addAll(dirInfo.getSubDirectories());
+          pathsToList.addAll(dirInfo.getSubDirectories().stream()
+              .map(path -> new SerializablePath(new CachingPath(path.toUri())))
+              .collect(Collectors.toList()));
         }
       }
     }

Reply via email to