You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/04/19 09:13:13 UTC

[GitHub] [iceberg] lcspinter opened a new pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

lcspinter opened a new pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494


   SparkTableUtil introduced some utility methods (`public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format, PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig, NameMapping mapping`) which are used to generate `DataFile`s when importing files from existing Spark table into Iceberg table. These utility methods can be useful when migrating Hive table to Iceberg table, therefore it would make sense to extract them to the core module. 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] lcspinter commented on a change in pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
lcspinter commented on a change in pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#discussion_r616607521



##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     * <p>
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     * <p>
+     * Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but not file footers,
+     * will not be populated.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @param mapping a name mapping
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig,
+                                             NameMapping mapping) {
+    if (format.contains("avro")) {
+      return listAvroPartition(partitionKeys, uri, spec, conf);
+    } else if (format.contains("parquet")) {
+      return listParquetPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else if (format.contains("orc")) {
+      return listOrcPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else {
+      throw new UnsupportedOperationException("Unknown partition format: " + format);
+    }
+  }
+
+  private static List<DataFile> listAvroPartition(Map<String, String> partitionPath, String partitionUri,
+                                                  PartitionSpec spec, Configuration conf) {
+    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 -> {
+                  // Avro file statistics cannot be calculated without reading the file.

Review comment:
       @aokolnychyi I changed back to the original value (-1L) and I will open a separate PR to address this issue. 
   With -1 set as rowCount will result in [failure](https://github.com/apache/iceberg/blob/a79de571860a290f6e96ac562d616c9c6be2071e/core/src/main/java/org/apache/iceberg/DataFiles.java#L288) when calling DataFiles.Builder.build(). I checked the associated test suite `TestSparkTableUtil`, and it seems that the import of spark tables having data files in avro file format [was not tested](https://github.com/apache/iceberg/blob/a79de571860a290f6e96ac562d616c9c6be2071e/spark2/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java#L139) at all.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-827669164


   @lcspinter I have no problem keeping the behavior of with the -1 as is for now even if it's broken. We should fix it when we get real working avro tests, which we probably can do in the integration suite I setup. It shouldn't have the same classloader baggage our normal test classpath has.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-822369509


   @aokolnychyi @RussellSpitzer can you please take a look as well, since it touches the Spark/Data modules?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] lcspinter commented on a change in pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
lcspinter commented on a change in pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#discussion_r616607521



##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     * <p>
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     * <p>
+     * Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but not file footers,
+     * will not be populated.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @param mapping a name mapping
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig,
+                                             NameMapping mapping) {
+    if (format.contains("avro")) {
+      return listAvroPartition(partitionKeys, uri, spec, conf);
+    } else if (format.contains("parquet")) {
+      return listParquetPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else if (format.contains("orc")) {
+      return listOrcPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else {
+      throw new UnsupportedOperationException("Unknown partition format: " + format);
+    }
+  }
+
+  private static List<DataFile> listAvroPartition(Map<String, String> partitionPath, String partitionUri,
+                                                  PartitionSpec spec, Configuration conf) {
+    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 -> {
+                  // Avro file statistics cannot be calculated without reading the file.

Review comment:
       @aokolnychyi I changed back to the original value (-1L) and I will open a separate PR to address this issue. 
   With -1 set as rowCount will result in [failure|https://github.com/apache/iceberg/blob/a79de571860a290f6e96ac562d616c9c6be2071e/core/src/main/java/org/apache/iceberg/DataFiles.java#L288] when calling DataFiles.Builder.build(). I checked the associated test suite `TestSparkTableUtil`, and it seems that the import of spark tables having data files in avro file format [was not tested|https://github.com/apache/iceberg/blob/a79de571860a290f6e96ac562d616c9c6be2071e/spark2/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java#L139] at all.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#discussion_r615725072



##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {

Review comment:
       Do we want to keep this as a generic utility class for all sorts of data helper operations? Or do we want this to contain only migration related helper methods, e.g. `TableMigrationUtil`?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-822824869


   Looks good to me aside from naming of the utility class and the formatting issues đź‘Ť 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#discussion_r616178188



##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,

Review comment:
       I think it should be either `partition` like before or `partitionKey` as Iceberg `PartitionKey` represents a struct of partition values. 

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *

Review comment:
       Looks like the overall formatting of the doc is off by one space.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -281,155 +270,10 @@ private SparkTableUtil() {
   public static List<DataFile> listPartition(SparkPartition partition, PartitionSpec spec,

Review comment:
       Shall we deprecate these methods?

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**

Review comment:
       Doc formatting here too.

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,

Review comment:
       Actually, do we need this method without `NameMapping`? I think it has no usages now and we should definitely encourage a name mapping to be present during imports.

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     * <p>
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     * <p>
+     * Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but not file footers,
+     * will not be populated.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @param mapping a name mapping
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,

Review comment:
       Same for `partitionKeys` here.

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     * <p>
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     * <p>
+     * Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but not file footers,
+     * will not be populated.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @param mapping a name mapping
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig,
+                                             NameMapping mapping) {
+    if (format.contains("avro")) {
+      return listAvroPartition(partitionKeys, uri, spec, conf);
+    } else if (format.contains("parquet")) {
+      return listParquetPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else if (format.contains("orc")) {
+      return listOrcPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else {
+      throw new UnsupportedOperationException("Unknown partition format: " + format);
+    }
+  }
+
+  private static List<DataFile> listAvroPartition(Map<String, String> partitionPath, String partitionUri,
+                                                  PartitionSpec spec, Configuration conf) {
+    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 -> {
+                  // Avro file statistics cannot be calculated without reading the file.

Review comment:
       This does not seem to match the existing logic. If we want to make this change, can we do that in a separate PR?
   Also, won't it break the existing filtering logic?

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {

Review comment:
       I agree with @marton-bod. The currently proposed name is not descriptive. I think `TableMigrationUtil` fits better.

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     * <p>
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     * <p>
+     * Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but not file footers,
+     * will not be populated.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @param mapping a name mapping
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig,
+                                             NameMapping mapping) {
+    if (format.contains("avro")) {
+      return listAvroPartition(partitionKeys, uri, spec, conf);
+    } else if (format.contains("parquet")) {
+      return listParquetPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else if (format.contains("orc")) {
+      return listOrcPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else {
+      throw new UnsupportedOperationException("Unknown partition format: " + format);
+    }
+  }
+
+  private static List<DataFile> listAvroPartition(Map<String, String> partitionPath, String partitionUri,
+                                                  PartitionSpec spec, Configuration conf) {
+    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 -> {
+                  // Avro file statistics cannot be calculated without reading the file.
+                  // Setting the rowCount to 0 is just a workaround so that the DataFiles.Builder.build() doesn't fail.
+                Metrics metrics = new Metrics(0L, null, null, null);
+                String partitionKey = spec.fields().stream()
+                          .map(PartitionField::name)

Review comment:
       Iceberg normally uses just 4 spaces for continued indentation. Can we format this class accordingly?

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     * <p>
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     * <p>
+     * Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but not file footers,
+     * will not be populated.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @param mapping a name mapping
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig,
+                                             NameMapping mapping) {
+    if (format.contains("avro")) {
+      return listAvroPartition(partitionKeys, uri, spec, conf);
+    } else if (format.contains("parquet")) {
+      return listParquetPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else if (format.contains("orc")) {
+      return listOrcPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else {
+      throw new UnsupportedOperationException("Unknown partition format: " + format);
+    }
+  }
+
+  private static List<DataFile> listAvroPartition(Map<String, String> partitionPath, String partitionUri,
+                                                  PartitionSpec spec, Configuration conf) {
+    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 -> {
+                  // Avro file statistics cannot be calculated without reading the file.
+                  // Setting the rowCount to 0 is just a workaround so that the DataFiles.Builder.build() doesn't fail.
+                Metrics metrics = new Metrics(0L, null, null, null);
+                String partitionKey = spec.fields().stream()
+                          .map(PartitionField::name)
+                          .map(name -> String.format("%s=%s", name, partitionPath.get(name)))
+                          .collect(Collectors.joining("/"));
+
+                return DataFiles.builder(spec)
+                          .withPath(stat.getPath().toString())
+                          .withFormat("avro")
+                          .withFileSizeInBytes(stat.getLen())
+                          .withMetrics(metrics)
+                          .withPartitionPath(partitionKey)
+                          .build();
+
+              }).collect(Collectors.toList());
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to list files in partition: " + partitionUri, e);
+    }
+  }
+
+  private static List<DataFile> listParquetPartition(Map<String, String> partitionPath, String partitionUri,
+                                                     PartitionSpec spec, Configuration conf,
+                                                     MetricsConfig metricsSpec, NameMapping mapping) {
+    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 -> {
+                Metrics metrics;
+                try {
+                  ParquetMetadata metadata = ParquetFileReader.readFooter(conf, stat);
+                  metrics = ParquetUtil.footerMetrics(metadata, Stream.empty(), metricsSpec, mapping);
+                } catch (IOException e) {
+                  throw new RuntimeException("Unable to read the footer of the parquet file: " +
+                              stat.getPath(), e);
+                }
+                String partitionKey = spec.fields().stream()
+                          .map(PartitionField::name)
+                          .map(name -> String.format("%s=%s", name, partitionPath.get(name)))
+                          .collect(Collectors.joining("/"));
+
+                return DataFiles.builder(spec)
+                          .withPath(stat.getPath().toString())
+                          .withFormat("parquet")
+                          .withFileSizeInBytes(stat.getLen())
+                          .withMetrics(metrics)
+                          .withPartitionPath(partitionKey)
+                          .build();
+              }).collect(Collectors.toList());
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to list files in partition: " + partitionUri, e);
+    }
+  }
+
+  private static List<DataFile> listOrcPartition(Map<String, String> partitionPath, String partitionUri,
+                                                 PartitionSpec spec, Configuration conf,
+                                                 MetricsConfig metricsSpec, NameMapping mapping) {
+    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 -> {
+                Metrics metrics = OrcMetrics.fromInputFile(HadoopInputFile.fromPath(stat.getPath(), conf),
+                          metricsSpec, mapping);

Review comment:
       formatting

##########
File path: data/src/main/java/org/apache/iceberg/data/DataUtil.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iceberg.data;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+
+public class DataUtil {
+
+  private DataUtil() {
+  }
+
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     *
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig) {
+    return listPartition(partitionKeys, uri, format, spec, conf, metricsConfig, null);
+  }
+
+    /**
+     * Returns the data files in a partition by listing the partition location.
+     * <p>
+     * For Parquet and ORC partitions, this will read metrics from the file footer. For Avro partitions,
+     * metrics are set to null.
+     * <p>
+     * Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but not file footers,
+     * will not be populated.
+     *
+     * @param partitionKeys partition key, e.g., "a=1/b=2"
+     * @param uri partition location URI
+     * @param format partition format, avro, parquet or orc
+     * @param spec a partition spec
+     * @param conf a Hadoop conf
+     * @param metricsConfig a metrics conf
+     * @param mapping a name mapping
+     * @return a List of DataFile
+     */
+  public static List<DataFile> listPartition(Map<String, String> partitionKeys, String uri, String format,
+                                             PartitionSpec spec, Configuration conf, MetricsConfig metricsConfig,
+                                             NameMapping mapping) {
+    if (format.contains("avro")) {
+      return listAvroPartition(partitionKeys, uri, spec, conf);
+    } else if (format.contains("parquet")) {
+      return listParquetPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else if (format.contains("orc")) {
+      return listOrcPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
+    } else {
+      throw new UnsupportedOperationException("Unknown partition format: " + format);
+    }
+  }
+
+  private static List<DataFile> listAvroPartition(Map<String, String> partitionPath, String partitionUri,
+                                                  PartitionSpec spec, Configuration conf) {
+    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 -> {
+                  // Avro file statistics cannot be calculated without reading the file.
+                  // Setting the rowCount to 0 is just a workaround so that the DataFiles.Builder.build() doesn't fail.
+                Metrics metrics = new Metrics(0L, null, null, null);
+                String partitionKey = spec.fields().stream()
+                          .map(PartitionField::name)
+                          .map(name -> String.format("%s=%s", name, partitionPath.get(name)))
+                          .collect(Collectors.joining("/"));
+
+                return DataFiles.builder(spec)
+                          .withPath(stat.getPath().toString())
+                          .withFormat("avro")
+                          .withFileSizeInBytes(stat.getLen())
+                          .withMetrics(metrics)
+                          .withPartitionPath(partitionKey)
+                          .build();
+
+              }).collect(Collectors.toList());
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to list files in partition: " + partitionUri, e);
+    }
+  }
+
+  private static List<DataFile> listParquetPartition(Map<String, String> partitionPath, String partitionUri,
+                                                     PartitionSpec spec, Configuration conf,
+                                                     MetricsConfig metricsSpec, NameMapping mapping) {
+    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 -> {
+                Metrics metrics;
+                try {
+                  ParquetMetadata metadata = ParquetFileReader.readFooter(conf, stat);
+                  metrics = ParquetUtil.footerMetrics(metadata, Stream.empty(), metricsSpec, mapping);
+                } catch (IOException e) {
+                  throw new RuntimeException("Unable to read the footer of the parquet file: " +
+                              stat.getPath(), e);

Review comment:
       nit: formatting, should be 4 spaces for continued indentation




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
flyrain commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-822913380


   Other than the naming and formatting issue, should it be in package org.apache.iceberg.util? Does it make sense to put them in class `PartitionUtil`?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-825059274


   @aokolnychyi @RussellSpitzer Thanks for reviewing this!
   As I see it now, the only outstanding thing is deprecating this method and otherwise good to go? https://github.com/apache/iceberg/pull/2494/commits/3d499cda445adbaf7c6ce7a529169a2d256e5159#diff-e9befa3e2e58983fca27df05bd7d62b673c327261522e3894bb5fc6d4800f17dR270
   Thank you! cc @lcspinter 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer merged pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer merged pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] lcspinter edited a comment on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
lcspinter edited a comment on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-826703041


   @RussellSpitzer @aokolnychyi I've marked the methods as deprecated. 
   What is left if to solve this [issue](https://github.com/apache/iceberg/pull/2494#discussion_r616607521) in a separate PR. With rowCount set to 0, the hive table migration works perfectly, but I'm not sure how will affect other use cases. Do you happen to know some other solution? Thanks


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] lcspinter commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
lcspinter commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-822310871


   @marton-bod @pvary Could you please review this PR? Thank you


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-823075017


   My impression is that `PartitionUtil` is supposed to define operations on Iceberg partitions, whereas here we are dealing with listing Hive partitions so it might not be a good idea to mix the two. I'd prefer keeping this migration-related logic in this separate `TableMigrationUtil` class, especially if more methods will be added to it as migration efforts pick up pace.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-827669731


   I think everyone's given their +1's so i'm going to merge this. Thanks for the contribution @lcspinter !


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-823221895


   Yeah, because of classloader issues I was only able two write tests for parquet. Both orc and Avro are set to ignore in the procedure/action tests
   
   Sent from my iPhone
   
   > On Apr 20, 2021, at 6:48 AM, László Pintér ***@***.***> wrote:
   > 
   > 
   > @lcspinter commented on this pull request.
   > 
   > In data/src/main/java/org/apache/iceberg/data/DataUtil.java:
   > 
   > > +    } else if (format.contains("orc")) {
   > +      return listOrcPartition(partitionKeys, uri, spec, conf, metricsConfig, mapping);
   > +    } else {
   > +      throw new UnsupportedOperationException("Unknown partition format: " + format);
   > +    }
   > +  }
   > +
   > +  private static List<DataFile> listAvroPartition(Map<String, String> partitionPath, String partitionUri,
   > +                                                  PartitionSpec spec, Configuration conf) {
   > +    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 -> {
   > +                  // Avro file statistics cannot be calculated without reading the file.
   > @aokolnychyi I changed back to the original value (-1L) and I will open a separate PR to address this issue.
   > With -1 set as rowCount will result in [failure|https://github.com/apache/iceberg/blob/a79de571860a290f6e96ac562d616c9c6be2071e/core/src/main/java/org/apache/iceberg/DataFiles.java#L288] when calling DataFiles.Builder.build(). I checked the associated test suite TestSparkTableUtil, and it seems that the import of spark tables having data files in avro file format [was not tested|https://github.com/apache/iceberg/blob/a79de571860a290f6e96ac562d616c9c6be2071e/spark2/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java#L139] at all.
   > 
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub, or unsubscribe.
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] lcspinter commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
lcspinter commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-826703041


   @RussellSpitzer @aokolnychyi I've marked the methods as deprecated. 
   What is left if to solve this [issue] (https://github.com/apache/iceberg/pull/2494#discussion_r616607521) in a separate PR. With rowCount set to 0, the hive table migration works perfectly, but I'm not sure how will affect other use cases. Do you happen to know some other solution? Thanks


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] lcspinter commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
lcspinter commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-823211573


   I've uploaded a new PR with the requested formatting/renaming changes. 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-822764095


   Let me take a look in a few hours. 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2494: Core: Extract listPartition methods from SparkTableUtil into DataUtil class in core module.

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2494:
URL: https://github.com/apache/iceberg/pull/2494#issuecomment-822798267


   @RussellSpitzer @flyrain, could you take a look too?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org