RussellSpitzer commented on a change in pull request #3876:
URL: https://github.com/apache/iceberg/pull/3876#discussion_r789859109



##########
File path: data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java
##########
@@ -74,111 +74,151 @@ private TableMigrationUtil() {
   public static List<DataFile> listPartition(Map<String, String> partition, 
String uri, String format,
                                              PartitionSpec spec, Configuration 
conf, MetricsConfig metricsConfig,
                                              NameMapping mapping) {
+    int parallelism = conf.getInt(PARQUET_READ_PARALLELISM, 1);
     if (format.contains("avro")) {
-      return listAvroPartition(partition, uri, spec, conf);
+      return listAvroPartition(partition, uri, spec, conf, parallelism);
     } else if (format.contains("parquet")) {
-      return listParquetPartition(partition, uri, spec, conf, metricsConfig, 
mapping);
+      return listParquetPartition(partition, uri, spec, conf, metricsConfig, 
mapping, parallelism);
     } else if (format.contains("orc")) {
-      return listOrcPartition(partition, uri, spec, conf, metricsConfig, 
mapping);
+      return listOrcPartition(partition, uri, spec, conf, metricsConfig, 
mapping, parallelism);
     } else {
       throw new UnsupportedOperationException("Unknown partition format: " + 
format);
     }
   }
 
   private static List<DataFile> listAvroPartition(Map<String, String> 
partitionPath, String partitionUri,
-                                                  PartitionSpec spec, 
Configuration conf) {
+                                                  PartitionSpec spec, 
Configuration conf, int parallelism) {
     try {
       Path partition = new Path(partitionUri);
       FileSystem fs = partition.getFileSystem(conf);
-      return Arrays.stream(fs.listStatus(partition, HIDDEN_PATH_FILTER))
-          .filter(FileStatus::isFile)
-          .map(stat -> {
-            InputFile file = 
HadoopInputFile.fromLocation(stat.getPath().toString(), conf);
-            long rowCount = Avro.rowCount(file);
-            Metrics metrics = new Metrics(rowCount, null, null, null, null);
-            String partitionKey = spec.fields().stream()
+      List<FileStatus> fileStatus = Arrays.stream(fs.listStatus(partition, 
HIDDEN_PATH_FILTER))
+              .filter(FileStatus::isFile)
+              .collect(Collectors.toList());
+      DataFile[] datafiles = new DataFile[fileStatus.size()];
+      Tasks.Builder<Integer> task = Tasks.range(fileStatus.size())

Review comment:
       Couldn't we just accumulate the results in a concurrent structure?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to