aokolnychyi commented on code in PR #11146:
URL: https://github.com/apache/iceberg/pull/11146#discussion_r1769469167


##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -289,4 +289,21 @@ private static Schema 
lazyColumnProjection(TableScanContext context, Schema sche
   public ThisT metricsReporter(MetricsReporter reporter) {
     return newRefinedScan(table, schema, context.reportWith(reporter));
   }
+
+  /**
+   * Retrieves a list of column names based on the type of manifest content 
provided.
+   *
+   * @param content the manifest content type to scan.
+   * @return a list of column names corresponding to the specified manifest 
content type.
+   */
+  public static List<String> scanColumns(ManifestContent content) {

Review Comment:
   Does it have to be public or package-private would do it?



##########
core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.ThreadPools;
+
+public class PartitionStatsUtil {
+
+  private PartitionStatsUtil() {}
+
+  /**
+   * Computes the partition stats for the given snapshot of the table.
+   *
+   * @param table the table for which partition stats to be computed.
+   * @param snapshot the snapshot for which partition stats is computed.
+   * @return iterable {@link PartitionStats}
+   */
+  public static Iterable<PartitionStats> computeStats(Table table, Snapshot 
snapshot) {
+    Preconditions.checkArgument(table != null, "table cannot be null");
+    Preconditions.checkArgument(snapshot != null, "snapshot cannot be null");
+
+    StructType partitionType = Partitioning.partitionType(table);
+    if (partitionType.fields().isEmpty()) {
+      throw new UnsupportedOperationException(
+          "Computing partition stats for an unpartitioned table");
+    }
+
+    List<ManifestFile> manifestFiles = snapshot.allManifests(table.io());

Review Comment:
   Minor: Just `manifestFiles` -> `manifests` for shorter lines?



##########
core/src/main/java/org/apache/iceberg/PartitionStats.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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;
+
+public class PartitionStats implements StructLike {
+
+  private static final int STATS_COUNT = 12;
+
+  private PartitionData partition;
+  private int specId;
+  private long dataRecordCount;
+  private int dataFileCount;
+  private long totalDataFileSizeInBytes;
+  private long positionDeleteRecordCount;
+  private int positionDeleteFileCount;
+  private long equalityDeleteRecordCount;
+  private int equalityDeleteFileCount;
+  private long totalRecordCount;
+  private Long lastUpdatedAt; // null by default
+  private Long lastUpdatedSnapshotId; // null by default
+
+  public PartitionStats(PartitionData partition, int specId) {
+    this.partition = partition;
+    this.specId = specId;
+  }
+
+  public PartitionData partition() {
+    return partition;
+  }
+
+  public int specId() {
+    return specId;
+  }
+
+  public long dataRecordCount() {
+    return dataRecordCount;
+  }
+
+  public int dataFileCount() {
+    return dataFileCount;
+  }
+
+  public long totalDataFileSizeInBytes() {
+    return totalDataFileSizeInBytes;
+  }
+
+  public long positionDeleteRecordCount() {
+    return positionDeleteRecordCount;
+  }
+
+  public int positionDeleteFileCount() {
+    return positionDeleteFileCount;
+  }
+
+  public long equalityDeleteRecordCount() {
+    return equalityDeleteRecordCount;
+  }
+
+  public int equalityDeleteFileCount() {
+    return equalityDeleteFileCount;
+  }
+
+  public long totalRecordCount() {
+    return totalRecordCount;
+  }
+
+  public Long lastUpdatedAt() {
+    return lastUpdatedAt;
+  }
+
+  public Long lastUpdatedSnapshotId() {
+    return lastUpdatedSnapshotId;
+  }
+
+  /**
+   * Updates the partition stats from the data/delete file.
+   *
+   * @param file the ContentFile from the manifest entry.
+   * @param snapshot the snapshot corresponding to the live entry.
+   */
+  public void liveEntry(ContentFile<?> file, Snapshot snapshot) {
+    this.specId = Math.max(this.specId, file.specId());

Review Comment:
   This seems a bit suspicious. What about a simple sanity check? I think if 
spec IDs don't match, it means something went wrong as `PartitionStats` is 
initialized with the spec ID.
   
   Would something like this be sufficient?
   
   ```
   Preconditions.checkArgument(specId == file.specId(), "Spec IDs must match");
   ```



##########
core/src/main/java/org/apache/iceberg/PartitionStats.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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;
+
+public class PartitionStats implements StructLike {
+
+  private static final int STATS_COUNT = 12;
+
+  private PartitionData partition;
+  private int specId;
+  private long dataRecordCount;
+  private int dataFileCount;
+  private long totalDataFileSizeInBytes;
+  private long positionDeleteRecordCount;
+  private int positionDeleteFileCount;
+  private long equalityDeleteRecordCount;
+  private int equalityDeleteFileCount;
+  private long totalRecordCount;
+  private Long lastUpdatedAt; // null by default
+  private Long lastUpdatedSnapshotId; // null by default
+
+  public PartitionStats(PartitionData partition, int specId) {
+    this.partition = partition;
+    this.specId = specId;
+  }
+
+  public PartitionData partition() {
+    return partition;
+  }
+
+  public int specId() {
+    return specId;
+  }
+
+  public long dataRecordCount() {
+    return dataRecordCount;
+  }
+
+  public int dataFileCount() {
+    return dataFileCount;
+  }
+
+  public long totalDataFileSizeInBytes() {
+    return totalDataFileSizeInBytes;
+  }
+
+  public long positionDeleteRecordCount() {
+    return positionDeleteRecordCount;
+  }
+
+  public int positionDeleteFileCount() {
+    return positionDeleteFileCount;
+  }
+
+  public long equalityDeleteRecordCount() {
+    return equalityDeleteRecordCount;
+  }
+
+  public int equalityDeleteFileCount() {
+    return equalityDeleteFileCount;
+  }
+
+  public long totalRecordCount() {
+    return totalRecordCount;
+  }
+
+  public Long lastUpdatedAt() {
+    return lastUpdatedAt;
+  }
+
+  public Long lastUpdatedSnapshotId() {
+    return lastUpdatedSnapshotId;
+  }
+
+  /**
+   * Updates the partition stats from the data/delete file.
+   *
+   * @param file the ContentFile from the manifest entry.
+   * @param snapshot the snapshot corresponding to the live entry.
+   */
+  public void liveEntry(ContentFile<?> file, Snapshot snapshot) {
+    this.specId = Math.max(this.specId, file.specId());
+
+    switch (file.content()) {
+      case DATA:
+        this.dataRecordCount += file.recordCount();
+        this.dataFileCount += 1;
+        this.totalDataFileSizeInBytes += file.fileSizeInBytes();
+        break;
+      case POSITION_DELETES:
+        this.positionDeleteRecordCount += file.recordCount();
+        this.positionDeleteFileCount += 1;
+        break;
+      case EQUALITY_DELETES:
+        this.equalityDeleteRecordCount += file.recordCount();
+        this.equalityDeleteFileCount += 1;
+        break;
+      default:
+        throw new UnsupportedOperationException("Unsupported file content 
type: " + file.content());
+    }
+
+    if (snapshot != null) {
+      updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis());
+    }
+
+    // Note: Not computing the `TOTAL_RECORD_COUNT` for now as it needs 
scanning the data.
+  }
+
+  /**
+   * Updates the modified time and snapshot ID for the deleted manifest entry.
+   *
+   * @param snapshot the snapshot corresponding to the deleted manifest entry.
+   */
+  public void deletedEntry(Snapshot snapshot) {
+    if (snapshot != null) {
+      updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis());
+    }
+  }
+
+  /**
+   * Appends statistics from given entry to current entry.
+   *
+   * @param entry the entry from which statistics will be sourced.
+   */
+  public void appendStats(PartitionStats entry) {
+    this.specId = Math.max(specId, entry.specId);

Review Comment:
   Same comment as in `liveEntry`, would make it a precondition instead.



##########
core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Comparator;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+import org.apache.iceberg.types.Types;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Timeout;
+import org.openjdk.jmh.annotations.Warmup;
+
+@Fork(1)
+@State(Scope.Benchmark)
+@Warmup(iterations = 2)
+@Measurement(iterations = 5)
+@Timeout(time = 1000, timeUnit = TimeUnit.HOURS)
+@BenchmarkMode(Mode.SingleShotTime)
+public class PartitionStatsUtilBenchmark {
+
+  private static final Schema SCHEMA =
+      new Schema(
+          required(1, "c1", Types.IntegerType.get()),
+          optional(2, "c2", Types.StringType.get()),
+          optional(3, "c3", Types.StringType.get()));
+
+  protected static final PartitionSpec SPEC =
+      PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private String baseDir;
+
+  // Create 10k manifests
+  private static final int MANIFEST_COUNTER = 10000;
+
+  // each manifest with 100 partition values
+  private static final int PARTITION_PER_MANIFEST = 100;
+
+  // 20 data files per partition, which results in 2k data files per manifest
+  private static final int DATA_FILES_PER_PARTITION_COUNT = 20;
+
+  private Table table;
+
+  @Setup
+  public void setupBenchmark() {
+    baseDir =
+        Paths.get(new 
File(System.getProperty("java.io.tmpdir")).getAbsolutePath()).toString();
+    table = TestTables.create(new File(baseDir), "foo", SCHEMA, SPEC, 
SortOrder.unsorted(), 2);
+
+    IntStream.range(0, MANIFEST_COUNTER)
+        .forEach(
+            manifestCount -> {
+              AppendFiles appendFiles = table.newAppend();
+
+              IntStream.range(0, PARTITION_PER_MANIFEST)
+                  .forEach(
+                      partitionOrdinal -> {
+                        StructLike partition = 
TestHelpers.Row.of(partitionOrdinal);
+                        IntStream.range(0, DATA_FILES_PER_PARTITION_COUNT)
+                            .forEach(
+                                fileOrdinal ->
+                                    appendFiles.appendFile(
+                                        
FileGenerationUtil.generateDataFile(table, partition)));
+                      });
+
+              appendFiles.commit();
+            });
+  }
+
+  @TearDown
+  public void tearDownBenchmark() throws IOException {
+    if (baseDir != null) {

Review Comment:
   What about using `HadoopTables` like in `AppendBenchmark`? That way, we can 
simply call drop.



##########
core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.ThreadPools;
+
+public class PartitionStatsUtil {
+
+  private PartitionStatsUtil() {}
+
+  /**
+   * Computes the partition stats for the given snapshot of the table.
+   *
+   * @param table the table for which partition stats to be computed.
+   * @param snapshot the snapshot for which partition stats is computed.
+   * @return iterable {@link PartitionStats}
+   */
+  public static Iterable<PartitionStats> computeStats(Table table, Snapshot 
snapshot) {

Review Comment:
   Question: Do we anticipate to do more tricks in this method in the future by 
returning `Iterable`? I wonder whether it would be better to return 
`Collection` so that the size is always known to the caller.



##########
core/src/main/java/org/apache/iceberg/PartitionStats.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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;
+
+public class PartitionStats implements StructLike {
+
+  private static final int STATS_COUNT = 12;
+
+  private PartitionData partition;

Review Comment:
   Shall we use `StructLike` given that we no longer need to use this as key in 
`ConcurrentHashMap`. I think the computation should rely on `StructLikeMap` 
now. Then we won't need changes in `PartitionUtil` and the implementation would 
be generic.



##########
core/src/main/java/org/apache/iceberg/PartitionStats.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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;
+
+public class PartitionStats implements StructLike {

Review Comment:
   This class seems solid, just a few suggestions.



##########
core/src/main/java/org/apache/iceberg/PartitionStats.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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;
+
+public class PartitionStats implements StructLike {
+
+  private static final int STATS_COUNT = 12;
+
+  private PartitionData partition;
+  private int specId;
+  private long dataRecordCount;
+  private int dataFileCount;
+  private long totalDataFileSizeInBytes;
+  private long positionDeleteRecordCount;
+  private int positionDeleteFileCount;
+  private long equalityDeleteRecordCount;
+  private int equalityDeleteFileCount;
+  private long totalRecordCount;
+  private Long lastUpdatedAt; // null by default
+  private Long lastUpdatedSnapshotId; // null by default
+
+  public PartitionStats(PartitionData partition, int specId) {
+    this.partition = partition;
+    this.specId = specId;
+  }
+
+  public PartitionData partition() {
+    return partition;
+  }
+
+  public int specId() {
+    return specId;
+  }
+
+  public long dataRecordCount() {
+    return dataRecordCount;
+  }
+
+  public int dataFileCount() {
+    return dataFileCount;
+  }
+
+  public long totalDataFileSizeInBytes() {
+    return totalDataFileSizeInBytes;
+  }
+
+  public long positionDeleteRecordCount() {
+    return positionDeleteRecordCount;
+  }
+
+  public int positionDeleteFileCount() {
+    return positionDeleteFileCount;
+  }
+
+  public long equalityDeleteRecordCount() {
+    return equalityDeleteRecordCount;
+  }
+
+  public int equalityDeleteFileCount() {
+    return equalityDeleteFileCount;
+  }
+
+  public long totalRecordCount() {
+    return totalRecordCount;
+  }
+
+  public Long lastUpdatedAt() {
+    return lastUpdatedAt;
+  }
+
+  public Long lastUpdatedSnapshotId() {
+    return lastUpdatedSnapshotId;
+  }
+
+  /**
+   * Updates the partition stats from the data/delete file.
+   *
+   * @param file the ContentFile from the manifest entry.
+   * @param snapshot the snapshot corresponding to the live entry.
+   */
+  public void liveEntry(ContentFile<?> file, Snapshot snapshot) {
+    this.specId = Math.max(this.specId, file.specId());
+
+    switch (file.content()) {
+      case DATA:
+        this.dataRecordCount += file.recordCount();
+        this.dataFileCount += 1;
+        this.totalDataFileSizeInBytes += file.fileSizeInBytes();
+        break;
+      case POSITION_DELETES:
+        this.positionDeleteRecordCount += file.recordCount();
+        this.positionDeleteFileCount += 1;
+        break;
+      case EQUALITY_DELETES:
+        this.equalityDeleteRecordCount += file.recordCount();
+        this.equalityDeleteFileCount += 1;
+        break;
+      default:
+        throw new UnsupportedOperationException("Unsupported file content 
type: " + file.content());
+    }
+
+    if (snapshot != null) {
+      updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis());
+    }
+
+    // Note: Not computing the `TOTAL_RECORD_COUNT` for now as it needs 
scanning the data.
+  }
+
+  /**
+   * Updates the modified time and snapshot ID for the deleted manifest entry.
+   *
+   * @param snapshot the snapshot corresponding to the deleted manifest entry.
+   */
+  public void deletedEntry(Snapshot snapshot) {
+    if (snapshot != null) {
+      updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis());
+    }
+  }
+
+  /**
+   * Appends statistics from given entry to current entry.
+   *
+   * @param entry the entry from which statistics will be sourced.
+   */
+  public void appendStats(PartitionStats entry) {
+    this.specId = Math.max(specId, entry.specId);
+    this.dataRecordCount += entry.dataRecordCount;
+    this.dataFileCount += entry.dataFileCount;
+    this.totalDataFileSizeInBytes += entry.totalDataFileSizeInBytes;
+    this.positionDeleteRecordCount += entry.positionDeleteRecordCount;
+    this.positionDeleteFileCount += entry.positionDeleteFileCount;
+    this.equalityDeleteRecordCount += entry.equalityDeleteRecordCount;
+    this.equalityDeleteFileCount += entry.equalityDeleteFileCount;
+    this.totalRecordCount += entry.totalRecordCount;
+
+    if (entry.lastUpdatedAt != null) {
+      updateSnapshotInfo(entry.lastUpdatedSnapshotId, entry.lastUpdatedAt);
+    }
+  }
+
+  private void updateSnapshotInfo(long snapshotId, long updatedAt) {
+    if (lastUpdatedAt == null || (lastUpdatedAt < updatedAt)) {

Review Comment:
   Do we need `()` in the second branch? Can it be just `lastUpdatedAt < 
updatedAt`?



##########
core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.ThreadPools;
+
+public class PartitionStatsUtil {
+
+  private PartitionStatsUtil() {}
+
+  /**
+   * Computes the partition stats for the given snapshot of the table.
+   *
+   * @param table the table for which partition stats to be computed.
+   * @param snapshot the snapshot for which partition stats is computed.
+   * @return iterable {@link PartitionStats}
+   */
+  public static Iterable<PartitionStats> computeStats(Table table, Snapshot 
snapshot) {
+    Preconditions.checkArgument(table != null, "table cannot be null");
+    Preconditions.checkArgument(snapshot != null, "snapshot cannot be null");
+
+    StructType partitionType = Partitioning.partitionType(table);
+    if (partitionType.fields().isEmpty()) {
+      throw new UnsupportedOperationException(
+          "Computing partition stats for an unpartitioned table");
+    }
+
+    List<ManifestFile> manifestFiles = snapshot.allManifests(table.io());
+
+    ExecutorService executorService = ThreadPools.getWorkerPool();
+    List<Future<Map<PartitionData, PartitionStats>>> futures = 
Lists.newArrayList();
+    manifestFiles.forEach(
+        manifest -> {
+          Future<Map<PartitionData, PartitionStats>> future =
+              executorService.submit(
+                  () -> {
+                    Map<PartitionData, PartitionStats> localStatsMap = 
Maps.newHashMap();
+                    collectStats(table, manifest, partitionType, 
localStatsMap);

Review Comment:
   What about making `collectStats` return `Map` instead?



##########
core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.ThreadPools;
+
+public class PartitionStatsUtil {
+
+  private PartitionStatsUtil() {}
+
+  /**
+   * Computes the partition stats for the given snapshot of the table.
+   *
+   * @param table the table for which partition stats to be computed.
+   * @param snapshot the snapshot for which partition stats is computed.
+   * @return iterable {@link PartitionStats}
+   */
+  public static Iterable<PartitionStats> computeStats(Table table, Snapshot 
snapshot) {
+    Preconditions.checkArgument(table != null, "table cannot be null");
+    Preconditions.checkArgument(snapshot != null, "snapshot cannot be null");
+
+    StructType partitionType = Partitioning.partitionType(table);
+    if (partitionType.fields().isEmpty()) {
+      throw new UnsupportedOperationException(
+          "Computing partition stats for an unpartitioned table");
+    }
+
+    List<ManifestFile> manifestFiles = snapshot.allManifests(table.io());
+
+    ExecutorService executorService = ThreadPools.getWorkerPool();
+    List<Future<Map<PartitionData, PartitionStats>>> futures = 
Lists.newArrayList();

Review Comment:
   Will it be possible to use any of these options?
   
   - Our `Tasks` utility with `List<StructLikeMap<PartitionStats>>` result 
holder.
   - Our `Parallelterable`.
   
   Managing concurrent code is error-prone and harder to review. For instance, 
the current implementation won't stop even if one of the futures fails. To 
prevent subtle issues, I'd consider the options above.



-- 
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: issues-unsubscr...@iceberg.apache.org

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


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

Reply via email to