danny0405 commented on code in PR #9121:
URL: https://github.com/apache/hudi/pull/9121#discussion_r1252515100


##########
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##########
@@ -106,42 +107,33 @@ private List<String> 
getPartitionPathWithPathPrefix(String relativePathPrefix) t
       // TODO: Get the parallelism from HoodieWriteConfig
       int listingParallelism = Math.min(DEFAULT_LISTING_PARALLELISM, 
pathsToList.size());
 
-      // List all directories in parallel
+      // List all directories in parallel:
+      // if current dictionary contains PartitionMetadata, add it to result
+      // if current dictionary does not contain PartitionMetadata, add its 
subdirectory to queue to be processed.
       engineContext.setJobStatus(this.getClass().getSimpleName(), "Listing all 
partitions with prefix " + relativePathPrefix);
-      List<FileStatus> dirToFileListing = engineContext.flatMap(pathsToList, 
path -> {
+      // result below holds a list of pair. first entry in the pair optionally 
holds the deduced list of partitions.
+      // and second entry holds optionally a directory path to be processed 
further.
+      List<Pair<Option<String>, Option<Path>>> result = 
engineContext.flatMap(pathsToList, path -> {
         FileSystem fileSystem = path.getFileSystem(hadoopConf.get());
-        return Arrays.stream(fileSystem.listStatus(path));
+        if (HoodiePartitionMetadata.hasPartitionMetadata(fileSystem, path)) {
+          return 
Stream.of(Pair.of(Option.of(FSUtils.getRelativePartitionPath(new 
Path(datasetBasePath), path)), Option.empty()));
+        }
+        return Arrays.stream(fileSystem.listStatus(path, p -> {
+          try {
+            return fileSystem.isDirectory(p) && 
!p.getName().equals(HoodieTableMetaClient.METAFOLDER_NAME);
+          } catch (IOException e) {
+            // noop
+          }
+          return false;
+        })).map(status -> Pair.of(Option.empty(), 
Option.of(status.getPath())));
       }, listingParallelism);
       pathsToList.clear();
 
-      // if current dictionary contains PartitionMetadata, add it to result
-      // if current dictionary does not contain PartitionMetadata, add it to 
queue to be processed.
-      int fileListingParallelism = Math.min(DEFAULT_LISTING_PARALLELISM, 
dirToFileListing.size());
-      if (!dirToFileListing.isEmpty()) {
-        // result below holds a list of pair. first entry in the pair 
optionally holds the deduced list of partitions.
-        // and second entry holds optionally a directory path to be processed 
further.
-        engineContext.setJobStatus(this.getClass().getSimpleName(), 
"Processing listed partitions");
-        List<Pair<Option<String>, Option<Path>>> result = 
engineContext.map(dirToFileListing, fileStatus -> {
-          FileSystem fileSystem = 
fileStatus.getPath().getFileSystem(hadoopConf.get());
-          if (fileStatus.isDirectory()) {
-            if (HoodiePartitionMetadata.hasPartitionMetadata(fileSystem, 
fileStatus.getPath())) {
-              return Pair.of(Option.of(FSUtils.getRelativePartitionPath(new 
Path(datasetBasePath), fileStatus.getPath())), Option.empty());
-            } else if 
(!fileStatus.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) 
{
-              return Pair.of(Option.empty(), Option.of(fileStatus.getPath()));
-            }
-          } else if 
(fileStatus.getPath().getName().startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX))
 {
-            String partitionName = FSUtils.getRelativePartitionPath(new 
Path(datasetBasePath), fileStatus.getPath().getParent());
-            return Pair.of(Option.of(partitionName), Option.empty());
-          }
-          return Pair.of(Option.empty(), Option.empty());
-        }, fileListingParallelism);
-
-        partitionPaths.addAll(result.stream().filter(entry -> 
entry.getKey().isPresent()).map(entry -> entry.getKey().get())
-            .collect(Collectors.toList()));
+      partitionPaths.addAll(result.stream().filter(entry -> 
entry.getKey().isPresent()).map(entry -> entry.getKey().get())
+          .collect(Collectors.toList()));

Review Comment:
   good point, the code looks much simpler!



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