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 2022/07/13 11:05:10 UTC

[GitHub] [iceberg] nastra opened a new pull request, #5268: Initial ScanMetricsContext / ScanReporting support

nastra opened a new pull request, #5268:
URL: https://github.com/apache/iceberg/pull/5268

   The idea here is that we collect metrics during Table scans so that we
   can better understand what exactly happens during a Table scan.
   
   This is just an initial implementation that gathers/records a few metrics. The idea is to get early feedback on this approach.


-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922325662


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public CloseableSample startCloseable() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultCloseableSample(stopwatchRef, defaultTimeUnit, this);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      this.duration = this.duration.plus(amount, toChronoUnit(unit));
+      count.incrementAndGet();
+    }
+  }
+
+  @Override
+  public <T> T record(Supplier<T> supplier) {
+    Sample sample = start();
+    try {
+      return supplier.get();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public <T> T recordCallable(Callable<T> callable) throws Exception {
+    Sample sample = start();
+    try {
+      return callable.call();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public void record(Runnable runnable) {
+    Sample sample = start();
+    try {
+      runnable.run();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("defaultTimeUnit", defaultTimeUnit)
+        .add("count", count)
+        .add("duration", duration)
+        .add("current", stopwatchRef)
+        .toString();
+  }
+
+  private static class DefaultSample implements Sample {
+    private final AtomicReference<Stopwatch> stopwatchRef;
+    private final TimeUnit defaultTimeUnit;
+
+    private DefaultSample(AtomicReference<Stopwatch> stopwatchRef, TimeUnit defaultTimeUnit) {
+      this.stopwatchRef = stopwatchRef;
+      this.defaultTimeUnit = defaultTimeUnit;
+    }
+
+    public void stop(Timer timer) {
+      Preconditions.checkArgument(null != timer, "Timer must be non-null");
+      Stopwatch stopwatch = stopwatchRef.getAndSet(null);
+      Preconditions.checkState(null != stopwatch, "startTimer() was not called");

Review Comment:
   This class should own the stopwatch, so that there is no way for this to happen. When a sample is created, the stopwatch is also created and started. The only state that we care about is if `stop` is called multiple times on the same `Sample`.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922329566


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalScanDuration() {

Review Comment:
   This is tracking scan planning, not a scan. Values like this should be `totalPlanningDuration`.
   
   We also typically use `long` to return durations in milliseconds. Is there a reason to use `Duration` instead? Are millis not granular enough?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923014777


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {
+    /**
+     * Stops the timer and records the total duration up until {@link Timer#start()} was called.
+     */
+    void stop(Timer timer);
+
+    Sample NOOP = timer -> { };
+  }
+
+  interface CloseableSample extends AutoCloseable {

Review Comment:
   the only reason I figured this might be a good idea is the IDE complaining about `Timer#start()` being used without a `try-with-resources` statement everywhere, but I guess the API is less readable so it's probably better to make `Timed` extend `AutoCloseable`



##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {
+    /**
+     * Stops the timer and records the total duration up until {@link Timer#start()} was called.
+     */
+    void stop(Timer timer);

Review Comment:
   agreed, fixed



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923482066


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {

Review Comment:
   yes correct



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923486960


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {

Review Comment:
   this is tracked as part of `totalDataManifestsRead()`



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922337676


##########
api/src/test/java/org/apache/iceberg/io/TestCloseableIterable.java:
##########
@@ -82,4 +85,61 @@ public void testConcateWithEmptyIterables() {
         NoSuchElementException.class,
         () -> Iterables.getLast(concat5));
   }
+
+  @Test
+  public void testWithCompletionRunnable() throws IOException {
+    AtomicInteger completionCounter = new AtomicInteger(0);
+    List<Integer> items = Lists.newArrayList(1, 2, 3, 4, 5);
+    Assertions.assertThatThrownBy(() ->
+                    CloseableIterable.whenComplete(CloseableIterable.combine(items, () -> { }), null))
+            .isInstanceOf(NullPointerException.class)
+            .hasMessage("Cannot execute a null Runnable after completion");

Review Comment:
   Indentation is off. It should be 2 spaces per indent and 2 indents for continuation indents.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929180459


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   I don't think that we need to serialize timers or counters if we don't have separate `ScanReport` and `ScanMetrics` classes. Scan planning should happen only on the coordinator, driver, or planner node. The resulting tasks are serialized to workers, but the scan itself should not be (and `TableScanContext` is not `Serializable`).



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r930821585


##########
api/src/main/java/org/apache/iceberg/metrics/MetricsContext.java:
##########
@@ -126,15 +126,15 @@ public Timer timer(String name, TimeUnit unit) {
 
       @Override
       public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
-        return new Counter<T>() {
-          @Override
-          public void increment() {
-          }
-
-          @Override
-          public void increment(T amount) {
-          }
-        };
+        if (Integer.class.equals(type)) {

Review Comment:
   this is particuarly for the case where one would use `ScanMetrics NOOP = new ScanMetrics(MetricsContext.nullMetrics())`, which would then use `MetricsContext.nullMetrics()`. I don't think we'd want any access of `.value()` to throw an exception when `MetricsContext.nullMetrics()` is used, so we need NOOP counters for those (we can't make a generic NOOP counter, as `.value()` needs to return `0` for `Integer.class` and `0L` for `Long.class`). The below test verifies that exact behavior
   
   https://github.com/apache/iceberg/blob/3ed173b87909c221d66a6b09d70c8f8d4b562e51/api/src/test/java/org/apache/iceberg/metrics/TestScanReport.java#L60-L81



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929610469


##########
api/src/main/java/org/apache/iceberg/io/IterableUtils.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.io;
+
+import java.util.Iterator;
+import java.util.function.Predicate;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IterableUtils {
+
+  private IterableUtils() {
+  }
+
+  /**
+   * Counts the number of elements in the given {@link Iterable} by incrementing the {@link Counter}
+   * instance for each {@link Iterator#next()} call.
+   *
+   * @param counter The {@link Counter} instance to increment on each {@link Iterator#next()} call.
+   * @param iterable The underlying {@link Iterator} to count
+   * @param <T> The underlying type to be iterated.
+   * @return An {@link Iterable} that increments the given counter on each {@link Iterator#next()}
+   *     call.
+   */
+  public static <T> Iterable<T> count(Counter<?> counter, Iterable<T> iterable) {

Review Comment:
   Is this something we'd want to do as a follow-up? As it would require to change a few places and I didn't want to pollute this PR with semi-related 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922332191


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {

Review Comment:
   Is the intent to keep adding new metrics to this?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923205181


##########
core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java:
##########
@@ -34,7 +35,7 @@
 class IncrementalDataTableScan extends DataTableScan {
 
   IncrementalDataTableScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
-    super(ops, table, schema, context.useSnapshotId(null));
+    super(ops, table, schema, context.useSnapshotId(null), new LoggingScanReporter());

Review Comment:
   I haven't considered that, but that makes perfect sense. Updated the code



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r925859764


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   Following up on [this comment](https://github.com/apache/iceberg/pull/5268#discussion_r922334117), I don't think that this should produce -1.
   
   The reason why `count` returns an `Optional` is that implementations, like Hadoop counters, may not be able to return the final value. Those counters aren't appropriate here, because we need to get the value to build a scan report.
   
   There are a couple options to fix. One is to log an error and not send the scan report, so this method would return null. I'm not a big fan of that, or of throwing an exception because metrics collection should not fail scans.
   
   The fix that I think makes the most sense is to let the `ScanReporter` implementation handle all of this internally. There's not really a need to have separate classes for `ScanMetrics` and `ScanReport`, when the report just tries to get the final metrics values. Separating the two classes creates a situation where the Iceberg library needs to interpret the `ScanMetrics` implementation that is provided by the `ScanReporter`. Instead, I think it's simpler just to pass the `ScanMetrics` (which were created by the `ScanReporter`) back as the final report.
   
   I think removing `ScanReport` and having just `ScanMetrics` is a cleaner in general as well. I don't see a clear purpose for a separate `ScanReport`, and a simpler API is generally better. This would probably require a slight rename, since we would want to pass the table name, filter, and projection in the combined class. So `ScanMetrics` would change to `ScanInfo`? Or maybe use `ScanReport` for the combined class.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922376822


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -234,7 +255,13 @@ private <T> Iterable<CloseableIterable<T>> entries(
           manifest -> manifest.hasAddedFiles() || manifest.hasDeletedFiles());
     }
 
-    matchingManifests = Iterables.filter(matchingManifests, manifestPredicate::test);
+    matchingManifests = Iterables.filter(matchingManifests, manifest -> {

Review Comment:
   Rather than changing the filtering, why not make a new Iterable that updates a counter? Then you'd have less custom code:
   
   ```java
     class <T> Iterable<T> count(Counter counter, Iterable<T> iterable) {
       return () -> new Iterator<T> {
         Iterator<T> iter = iterable.iterator();
   
         public boolean hasNext() {
           return iter.hasNext();
         }
   
         public T next() {
           T next = iter.next();
           counter.increment();
           return next;
         }
       }
     }
   
     ...
   
     matchingManifests = count(scanMetrics.matchingDataManifests(), Iterables.filter(...));
   ```



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r925854574


##########
api/src/main/java/org/apache/iceberg/metrics/LoggingScanReporter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.metrics;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A default {@link ScanReporter} implementation that logs the {@link ScanReport} to the log file.
+ */
+public class LoggingScanReporter implements ScanReporter {
+  private static final Logger LOG = LoggerFactory.getLogger(LoggingScanReporter.class);
+
+  @Override
+  @SuppressWarnings("Slf4jConstantLogMessage")
+  public void reportScan(ScanReport scanReport) {
+    Preconditions.checkArgument(null != scanReport, "ScanReport must be non-null");
+    LOG.info("Ending table scan");
+    LOG.info(scanReport.toString());

Review Comment:
   Why use two info messages rather than one to report the result? I'd expect something like `"Completed scan planning: %s", report`



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923016976


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public CloseableSample startCloseable() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");

Review Comment:
   updated



##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public CloseableSample startCloseable() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());

Review Comment:
   updated



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923015957


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {
+    /**
+     * Stops the timer and records the total duration up until {@link Timer#start()} was called.
+     */
+    void stop(Timer timer);
+
+    Sample NOOP = timer -> { };
+  }
+
+  interface CloseableSample extends AutoCloseable {
+    @Override
+    void close();
+    CloseableSample NOOP = () -> { };
+  }
+
+  Timer NOOP = new Timer() {
+    @Override
+    public Sample start() {
+      return Sample.NOOP;
+    }
+
+    @Override
+    public CloseableSample startCloseable() {
+      return CloseableSample.NOOP;
+    }
+
+    @Override
+    public long count() {
+      return 0;
+    }
+
+    @Override
+    public Duration totalDuration() {
+      return Duration.ZERO;
+    }
+
+    @Override
+    public void record(long amount, TimeUnit unit) {
+    }
+
+    @Override
+    public void record(Runnable runnable) {

Review Comment:
   fixed



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r931938902


##########
api/src/main/java/org/apache/iceberg/io/IterableUtils.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.io;
+
+import java.util.Iterator;
+import java.util.function.Predicate;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IterableUtils {
+
+  private IterableUtils() {
+  }
+
+  /**
+   * Counts the number of elements in the given {@link Iterable} by incrementing the {@link Counter}
+   * instance for each {@link Iterator#next()} call.
+   *
+   * @param counter The {@link Counter} instance to increment on each {@link Iterator#next()} call.
+   * @param iterable The underlying {@link Iterator} to count
+   * @param <T> The underlying type to be iterated.
+   * @return An {@link Iterable} that increments the given counter on each {@link Iterator#next()}
+   *     call.
+   */
+  public static <T> Iterable<T> count(Counter<?> counter, Iterable<T> iterable) {

Review Comment:
   ok I removed `IterableUtils` and moved relevant things to `CloseableIterable`



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921369617


##########
api/src/main/java/org/apache/iceberg/metrics/LongCounter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.metrics;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A default {@link org.apache.iceberg.metrics.MetricsContext.Counter} implementation that uses a {@link Long} to count
+ * events.
+ */
+class LongCounter implements MetricsContext.Counter<Long> {
+  private final AtomicLong counter;
+
+  LongCounter() {
+    counter = new AtomicLong(0L);
+  }
+
+  @Override
+  public void increment() {
+    counter.incrementAndGet();
+  }
+
+  @Override
+  public void increment(Long amount) {
+    counter.addAndGet(amount);

Review Comment:
   Should we use `updateAndGet` with `Math.addExact` to avoid overflows?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921899422


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;

Review Comment:
   I like that idea. Would you like me to add this as part of this PR or rather later?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r934247268


##########
core/src/main/java/org/apache/iceberg/DeleteFileIndex.java:
##########
@@ -502,6 +509,9 @@ DeleteFileIndex build() {
         }
       }
 
+      if (null != globalDeletes) {
+        scanMetrics.indexedDeleteFiles().increment(globalDeletes.length);
+      }

Review Comment:
   good catch as I wasn't sure if that one is correct or not. I'm counting the entries that are processed in the first loop now



-- 
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-unsubscribe@iceberg.apache.org

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] nastra closed pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra closed pull request #5268: API/Core: Initial Table Scan Reporting support
URL: https://github.com/apache/iceberg/pull/5268


-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r933639025


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -325,6 +351,9 @@ private static CloseableIterable<FileScanTask> createFileScanTasks(
         entry -> {
           DataFile dataFile = entry.file().copy(ctx.shouldKeepStats());
           DeleteFile[] deleteFiles = ctx.deletes().forEntry(entry);
+          ctx.scanMetrics().totalFileSizeInBytes().increment(dataFile.fileSizeInBytes());

Review Comment:
   Should we have a similar metrics for total delete file size?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936860820


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  private Table createTableWithCustomRecords(String tableName) throws IOException {
+    Schema schema =
+        new Schema(
+            required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            tableName,
+            schema,
+            PartitionSpec.builderFor(schema).build(),
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(tbl, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(tbl, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(tbl, Arrays.asList(record4));
+    tbl.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    tbl.refresh();
+    return tbl;
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();
+    TableScan tableScan = tbl.newScan();
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-with-deletes");
+    assertThat(scanReport.snapshotId()).isEqualTo(2L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isEqualTo(30L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(20L);
+  }
+
+  @Test
+  public void multipleDataManifests() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "multiple-data-manifests",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
+    long snapshotIdOne = tbl.currentSnapshot().snapshotId();
+    tbl.newAppend().appendFile(FILE_C).appendFile(FILE_D).commit();
+    long snapshotIdTwo = tbl.currentSnapshot().snapshotId();
+
+    List<ManifestFile> manifests = tbl.currentSnapshot().dataManifests(tbl.io());
+
+    ManifestFile manifestOne =
+        writeManifest(
+            "manifest-file-1.avro",
+            manifestEntry(Status.EXISTING, snapshotIdOne, FILE_A),
+            manifestEntry(Status.EXISTING, snapshotIdOne, FILE_B));
+
+    ManifestFile manifestTwo =
+        writeManifest(
+            "manifest-file-2.avro",
+            manifestEntry(Status.EXISTING, snapshotIdTwo, FILE_C),
+            manifestEntry(Status.EXISTING, snapshotIdTwo, FILE_D));
+
+    RewriteManifests rewriteManifests = tbl.rewriteManifests();
+    manifests.forEach(rewriteManifests::deleteManifest);
+    rewriteManifests.addManifest(manifestOne).addManifest(manifestTwo);
+    rewriteManifests.commit();

Review Comment:
   There's no need to use `rewriteManifests`. Each fast append will create a new manifest file, so I'd just use that instead.
   
   We also need a test case that filters manifests and results in `skippedDataManifests` with a non-zero value.



-- 
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-unsubscribe@iceberg.apache.org

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] danielcweeks commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r930487486


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int resultDataFiles;
+  private final int totalDataManifests;
+  private final int totalDeleteManifests;
+  private final int scannedDataManifests;
+  private final int scannedDeleteManifests;
+  private final int totalDataFiles;
+  private final int totalDeleteFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeInBytes;
+  private final int indexedDeleteFiles;
+  private final int resultDeleteFiles;
+  private final int skippedDataManifests;
+  private final int skippedDeleteManifests;
+  private final int skippedDataFiles;
+  private final int skippedDeleteFiles;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int resultDataFiles, int totalDataManifests, int totalDeleteManifests, int scannedDataManifests,
+      int scannedDeleteManifests, int totalDataFiles, int totalDeleteFiles, Duration totalPlanningDuration,
+      long totalFileSizeInBytes, int indexedDeleteFiles, int resultDeleteFiles,
+      int skippedDataManifests, int skippedDeleteManifests,
+      int skippedDataFiles, int skippedDeleteFiles) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.resultDataFiles = resultDataFiles;

Review Comment:
   I had a conversation with Ryan about what to do with ScanReport and ScanMetrics.  I think we actually want to remove the final values from the scan report here.  I get that will make it easy to serialize a pojo result, but there's a separation of concerns that I think we can achieve rather easily.
   
   Rather than collect all of the metric results in the ScanReport, just keep a reference to the ScanMetrics with the actual counters/timers.  This will allow the ScanReporter to manage the structure of the response.
   
   For example:
   
   ```java
   ScanReport:
     - table identifier
     - snapshot id
     - filter
     - projection
     - ScanMetrics:
         - <various metrics>
   ```
   
   The scan reporter can then construct the payload dynamically based off of the Scan report (espcially if we make a way to iterate over the metrics):
   
   ```json
   {
     "table-identifier": "a.b.c",
     "snapshot-id": 123,
     "filter": "foo=bar",
     "projection": "c1, c2",
     "metrics": {
       "<name>": {
         "type": "counter",   # we probably type metric type info and 
         "value": "val"
       },
       "<name>": {
         "type": "timer",
         "value": "val",
         "histogram": { "p50":  213, "p99":  18983 },
         "min": 23,
         "max": 20199
       }
     }
   }
   ```
   
   This allows us to separate the way we track metrics and the way we construct the payload so that if a future metric (say from micrometer) has a histogram we could then just update the reporter without have to create a new object structure for the values in ScanReport.  As you can see from what the json might look like, the metric info can get more complicated than we probably want to track here. 



##########
api/src/main/java/org/apache/iceberg/metrics/MetricsContext.java:
##########
@@ -126,15 +126,15 @@ public Timer timer(String name, TimeUnit unit) {
 
       @Override
       public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
-        return new Counter<T>() {
-          @Override
-          public void increment() {
-          }
-
-          @Override
-          public void increment(T amount) {
-          }
-        };
+        if (Integer.class.equals(type)) {

Review Comment:
   Not sure I follow why we need these NOOP implementations.  Seems like once you create a counter, you don't really need to know if it's a long/int/etc when using it.



##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   We might want to consider pulling this into the MetricsContext interface similar to `Counter` or at least expose how you create timers through that interface so that they can be named (this is necessary if you're delegating to a framework like micrometer).



##########
api/src/main/java/org/apache/iceberg/metrics/LoggingScanReporter.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.metrics;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A default {@link ScanReporter} implementation that logs the {@link ScanReport} to the log file.
+ */
+public class LoggingScanReporter implements ScanReporter {
+  private static final Logger LOG = LoggerFactory.getLogger(LoggingScanReporter.class);
+
+  @Override
+  @SuppressWarnings("Slf4jConstantLogMessage")
+  public void reportScan(ScanReport scanReport) {
+    Preconditions.checkArgument(null != scanReport, "Invalid scan report: null");
+    LOG.info("Completed scan planning: {}", scanReport);

Review Comment:
   If we update the ScanReport, we can still use this as long as the `ScanReport::toString` just iterates over the metrics and reports the value.



##########
api/src/main/java/org/apache/iceberg/io/CloseableIterable.java:
##########
@@ -75,6 +76,34 @@ public CloseableIterator<E> iterator() {
     };
   }
 
+  /**
+   * Will run the given runnable when {@link CloseableIterable#close()} has been called.
+   *
+   * @param iterable             The underlying {@link CloseableIterable} to iterate over
+   * @param onCompletionRunnable The runnable to run after the underlying iterable was closed
+   * @param <E>                  The type of der underlying iterable

Review Comment:
   "der" -> "the"



##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   @rdblue I looked into removing `Optional` with @nastra for counter and I believe that it makes sense.  For the hadoop case, we should probably throw unsupported or just adapt the implementation return the value from the file system statistics (which is possible, but would not currently be used) and then not have a default implementation for `count()`, which makes more sense to me.
   
   This avoids the question about `-1` vs. `Optional`.  I feel that if a counter is initialized, it should return 0 even if unused (as opposed to -1 or not present, which doesn't really have meaning in a counter context).
   
   @nastra said he would create a separate PR for this.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r931586411


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final ScanMetrics scanMetrics;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      ScanMetrics scanMetrics) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.scanMetrics = scanMetrics;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public ScanMetrics scanMetrics() {
+    return scanMetrics;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("scanMetrics", scanMetrics)
+        .toString();
+  }
+
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String newTableName) {
+      this.tableName = newTableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long newSnapshotId) {
+      this.snapshotId = newSnapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression newFilter) {
+      this.filter = newFilter;
+      return this;
+    }
+
+    public Builder withProjection(Schema newProjection) {
+      this.projection = newProjection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics newScanMetrics) {

Review Comment:
   Shouldn't this be created by the builder, rather than passed in?
   
   I would expect `ScanReporter` to have a method, `newReportBuilder()` that returns `new ScanReport.Builder()` so that the `TableScan` can add data to the report and then get `ScanMetrics` to accumulate metrics. For the builder, I think metrics would be configured using `withMetricsContext` to set a custom metrics context that `ScanMetrics` is based on.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936431265


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    TableScan tableScan = table.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();
+    TableScan tableScan = tbl.newScan();
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-with-deletes");
+    assertThat(scanReport.snapshotId()).isEqualTo(2L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);

Review Comment:
   added this as described in https://github.com/apache/iceberg/pull/5268#discussion_r936430695



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936898059


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  private Table createTableWithCustomRecords(String tableName) throws IOException {
+    Schema schema =
+        new Schema(
+            required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            tableName,
+            schema,
+            PartitionSpec.builderFor(schema).build(),
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(tbl, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(tbl, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(tbl, Arrays.asList(record4));
+    tbl.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    tbl.refresh();
+    return tbl;
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();
+    TableScan tableScan = tbl.newScan();
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-with-deletes");
+    assertThat(scanReport.snapshotId()).isEqualTo(2L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isEqualTo(30L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(20L);
+  }
+
+  @Test
+  public void multipleDataManifests() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "multiple-data-manifests",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
+    long snapshotIdOne = tbl.currentSnapshot().snapshotId();
+    tbl.newAppend().appendFile(FILE_C).appendFile(FILE_D).commit();
+    long snapshotIdTwo = tbl.currentSnapshot().snapshotId();
+
+    List<ManifestFile> manifests = tbl.currentSnapshot().dataManifests(tbl.io());
+
+    ManifestFile manifestOne =
+        writeManifest(
+            "manifest-file-1.avro",
+            manifestEntry(Status.EXISTING, snapshotIdOne, FILE_A),
+            manifestEntry(Status.EXISTING, snapshotIdOne, FILE_B));
+
+    ManifestFile manifestTwo =
+        writeManifest(
+            "manifest-file-2.avro",
+            manifestEntry(Status.EXISTING, snapshotIdTwo, FILE_C),
+            manifestEntry(Status.EXISTING, snapshotIdTwo, FILE_D));
+
+    RewriteManifests rewriteManifests = tbl.rewriteManifests();
+    manifests.forEach(rewriteManifests::deleteManifest);
+    rewriteManifests.addManifest(manifestOne).addManifest(manifestTwo);
+    rewriteManifests.commit();

Review Comment:
   updated. Also the second part in that test method has `assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(1)`



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r925858279


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")

Review Comment:
   What is this suppressing?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926541887


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   one of the reasons I added this distinction between `ScanMetrics` and the final `ScanReport` is so that we wouldn't need to serialize Counters/Timers. Additionally, it seemed cleaner to me to have metrics that are being updated wrapped in one class, where we would derive a final `ScanReport` from that is immutable (well kind-of). 
   The `ScanReport` is just a  Data holder that is serializable, but it doesn't carry the baggage of whatever a particular `Counter`/`Timer` implementation would have.
   
   I understand the reasoning behind making `Counter#count()` return an `Optional` but I was actually wondering whether it wouldn't be better to make this return an actual value. It seems overly restrictive to have this be an `Optional` for a special case, such as Hadoop. Also you generally don't see such a restriction in the Counter API of Micrometer. Maybe in such a case the `Counters` that are being produced by `HadoopMetricsContext` would just return -1 for `count()` if they can't fetch whatever they are counting from `FileSystem.Statistics`.
   @danielcweeks thoughts on this?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923147758


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalScanDuration() {
+    return totalScanDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalScanDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFilesCount().count().orElse(-1),

Review Comment:
   -1 means here that this wasn't tracked/available, but 0 means that this was tracked and was actually 0



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929189947


##########
api/src/main/java/org/apache/iceberg/io/IterableUtils.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.io;
+
+import java.util.Iterator;
+import java.util.function.Predicate;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IterableUtils {
+
+  private IterableUtils() {
+  }
+
+  /**
+   * Counts the number of elements in the given {@link Iterable} by incrementing the {@link Counter}
+   * instance for each {@link Iterator#next()} call.
+   *
+   * @param counter The {@link Counter} instance to increment on each {@link Iterator#next()} call.
+   * @param iterable The underlying {@link Iterator} to count
+   * @param <T> The underlying type to be iterated.
+   * @return An {@link Iterable} that increments the given counter on each {@link Iterator#next()}
+   *     call.
+   */
+  public static <T> Iterable<T> count(Counter<?> counter, Iterable<T> iterable) {

Review Comment:
   It would be safer in general if these methods were moved to `CloseableIterable`. Then if the argument is `Iterable` we can wrap it with a noop close and if the argument is `CloseableIterable` we handle it correctly. Adding versions that dont' support close could easily cause trouble with later refactors.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#issuecomment-1185557923

   I applied all the review feedback and splitted out the introduction of a `DefaultMetricsContext` and the `Timer` API into https://github.com/apache/iceberg/pull/5286. Once https://github.com/apache/iceberg/pull/5286 is merged, I'll rebase this PR so that it only contains Table Scan related 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922333578


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {

Review Comment:
   I think we want a metric for total data manifests as well so that we can track the % of data manifests that were scanned.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922325067


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public CloseableSample startCloseable() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());

Review Comment:
   This stopwatch should be owned by the `Sample` and not shared.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922335917


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReporter.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.metrics;
+
+import java.util.Map;
+import org.apache.iceberg.metrics.ScanReport.ScanMetrics;
+
+/**
+ * This interface defines the basic API for a Table Scan Reporter that can be used to collect and report different
+ * metrics during a Table scan. The general usage of the Scan Reporter is as following:
+ * <ul>
+ *   <li>startScan() to indicate that this is a new Scan so that metrics (and other things) can be initialized</li>
+ *   <li>use scanMetrics() to add recorded metrics during a Scan</li>
+ *   <li>reportScan() at the end of the Scan</li>
+ * </ul>
+ */
+public interface ScanReporter {
+
+  /**
+   * Indicates that a new Scan is about to be performed, where implementors should initialize the {@link ScanMetrics},
+   * which can then be used to add recorded metrics to during a Scan.
+   */
+  void startScan();
+
+  /**
+   * Indicates that a Scan is done by reporting a {@link ScanReport}. A {@link ScanReport} is usually directly derived
+   * from a {@link ScanMetrics} instance.
+   *
+   * @param scanReport The {@link ScanReport} to report.
+   */
+  void reportScan(ScanReport scanReport);
+
+  /**
+   * The current {@link ScanMetrics} that can be used during a table scan to add metrics to.

Review Comment:
   Why is a reporter tied to a single "current" `ScanMetrics`? Why not make this interface produce `ScanMetrics` and accept `ScanReport`?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922370524


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -163,20 +171,26 @@ public CloseableIterable<FileScanTask> planFiles() {
     });
 
     DeleteFileIndex deleteFiles = deleteIndexBuilder.build();
-
     boolean dropStats = ManifestReader.dropStats(dataFilter, columns);
     if (!deleteFiles.isEmpty()) {
       select(ManifestReader.withStatsColumns(columns));
     }
 
     Iterable<CloseableIterable<FileScanTask>> tasks = entries((manifest, entries) -> {
+      scanMetrics.addedDataFiles().increment(manifest.addedFilesCount());
+      scanMetrics.deletedDataFiles().increment(manifest.deletedFilesCount());
+      scanMetrics.totalDataManifestsReadCount().increment();
       int specId = manifest.partitionSpecId();
       PartitionSpec spec = specsById.get(specId);
       String schemaString = SchemaParser.toJson(spec.schema());
       String specString = PartitionSpecParser.toJson(spec);
       ResidualEvaluator residuals = residualCache.get(specId);
-      return CloseableIterable.transform(entries, e -> new BaseFileScanTask(
-          e.file().copy(!dropStats), deleteFiles.forEntry(e), schemaString, specString, residuals));
+      return CloseableIterable.transform(entries, e -> {
+        scanMetrics.matchingDataFilesCount().increment();
+        scanMetrics.totalFileSizeInBytes().increment(e.file().fileSizeInBytes());

Review Comment:
   I think a more reliable place to gather these by wrapping the iterator returned by `planFiles`. That way you don't have to worry about future changes to code futher filtering the iterator that you're wrapping here.



##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -163,20 +171,26 @@ public CloseableIterable<FileScanTask> planFiles() {
     });
 
     DeleteFileIndex deleteFiles = deleteIndexBuilder.build();
-
     boolean dropStats = ManifestReader.dropStats(dataFilter, columns);
     if (!deleteFiles.isEmpty()) {
       select(ManifestReader.withStatsColumns(columns));
     }
 
     Iterable<CloseableIterable<FileScanTask>> tasks = entries((manifest, entries) -> {
+      scanMetrics.addedDataFiles().increment(manifest.addedFilesCount());
+      scanMetrics.deletedDataFiles().increment(manifest.deletedFilesCount());
+      scanMetrics.totalDataManifestsReadCount().increment();
       int specId = manifest.partitionSpecId();
       PartitionSpec spec = specsById.get(specId);
       String schemaString = SchemaParser.toJson(spec.schema());
       String specString = PartitionSpecParser.toJson(spec);
       ResidualEvaluator residuals = residualCache.get(specId);
-      return CloseableIterable.transform(entries, e -> new BaseFileScanTask(
-          e.file().copy(!dropStats), deleteFiles.forEntry(e), schemaString, specString, residuals));
+      return CloseableIterable.transform(entries, e -> {
+        scanMetrics.matchingDataFilesCount().increment();
+        scanMetrics.totalFileSizeInBytes().increment(e.file().fileSizeInBytes());

Review Comment:
   I think a more reliable place to gather these by wrapping the iterator returned by `planFiles`. That way you don't have to worry about future changes to code further filtering the iterator that you're wrapping here.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921357778


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;

Review Comment:
   An alternative that I tend to like is to have a custom `Task` interface rather than using `Callable` that allows you to specify the exception type:
   
   ```java
     public interface TimedTask<R, E extends Exception> {
       R call() throws E;
     }
   ```
   
   That way, you can customize the exception type and don't need an outer `try` block if it only throws `RuntimeException`.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921350766


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();

Review Comment:
   What about using just `start` and `stop`? Isn't it clear that these are timer methods?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921366128


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.StringJoiner;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> current = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public void startTimer() {
+    boolean updated = current.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");

Review Comment:
   I think this is definitely going to be called in parallel, since we want to measure the time it takes to scan an individual manifest file, which happens in the worker pool.
   
   To do that, you may want `startTimer` to return an individual `TimedExecution` or something, similar to how `Iterable.iterator` returns an `Iterator` with its own state. When a `TimedExecution` has `stopTimer` called on it, it could then call `record`, which would work just fine because `record` is an `AtomicLong`.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922322301


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {
+    /**
+     * Stops the timer and records the total duration up until {@link Timer#start()} was called.
+     */
+    void stop(Timer timer);
+
+    Sample NOOP = timer -> { };
+  }
+
+  interface CloseableSample extends AutoCloseable {
+    @Override
+    void close();
+    CloseableSample NOOP = () -> { };
+  }
+
+  Timer NOOP = new Timer() {
+    @Override
+    public Sample start() {
+      return Sample.NOOP;
+    }
+
+    @Override
+    public CloseableSample startCloseable() {

Review Comment:
   I think this is probably unnecessary given that there's already a `start` method.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922318793


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;

Review Comment:
   I'd probably replace what's here with two variations, one that returns a value and one that has no return. Then you've covered all of the cases and can use method references to call with a Runnable, Supplier, or Callable.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922177849


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   I left the `AutoCloseable` changes in the commit mentioned above but left it out from https://github.com/apache/iceberg/pull/5286 for now. Let me know if you'd like to include it in https://github.com/apache/iceberg/pull/5286



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923206543


##########
core/src/main/java/org/apache/iceberg/BaseTableScan.java:
##########
@@ -44,8 +48,21 @@ protected BaseTableScan(TableOperations ops, Table table, Schema schema) {
     this(ops, table, schema, new TableScanContext());
   }
 
+  protected BaseTableScan(TableOperations ops, Table table, Schema schema, ScanReporter scanReporter) {
+    this(ops, table, schema, new TableScanContext(), scanReporter);
+  }
+
   protected BaseTableScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {

Review Comment:
   moved the `ScanReporter` into the `TableScanContext`, so this change isn't required anymore



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial ScanMetricsContext / ScanReporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r920988194


##########
api/src/main/java/org/apache/iceberg/metrics/ScanMetricsContext.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.metrics;
+
+import java.util.StringJoiner;
+
+public class ScanMetricsContext implements MetricsContext {

Review Comment:
   that makes perfect sense @danielcweeks, thanks. I'll adjust the implementation accordingly



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923614087


##########
core/src/main/java/org/apache/iceberg/BaseTableScan.java:
##########
@@ -116,9 +133,20 @@ public CloseableIterable<FileScanTask> planFiles() {
           ExpressionUtil.toSanitizedString(filter()));
 
       Listeners.notifyAll(new ScanEvent(table().name(), snapshot.snapshotId(), filter(), schema()));
-
-      return doPlanFiles();
-
+      scanReporter().startScan();
+      Timer.Sample sample = scanReporter().scanMetrics().totalScanDuration().start();

Review Comment:
   thanks for bringing this up. I haven't thought about that. I'll rewrite the implementation and add some tests to make sure that each independent table scan produce their own metrics/report



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926702828


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;

Review Comment:
   that makes perfect sense, 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r934275343


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {

Review Comment:
   that sounds good to me. I removed some counters and added one more test that verifies that the delete-related metrics produce what we'd expect



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936431668


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    TableScan tableScan = table.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();
+    TableScan tableScan = tbl.newScan();
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-with-deletes");
+    assertThat(scanReport.snapshotId()).isEqualTo(2L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isEqualTo(30L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(20L);
+  }
+
+  private DataFile writeParquetFile(Table tbl, List<GenericRecord> records) throws IOException {
+    File parquetFile = temp.newFile();
+    assertTrue(parquetFile.delete());
+    FileAppender<GenericRecord> appender =
+        Parquet.write(Files.localOutput(parquetFile))
+            .schema(tbl.schema())
+            .createWriterFunc(GenericParquetWriter::buildWriter)
+            .build();
+    try {
+      appender.addAll(records);
+    } finally {
+      appender.close();
+    }
+
+    PartitionKey partitionKey = new PartitionKey(tbl.spec(), tbl.schema());
+    partitionKey.partition(records.get(0));

Review Comment:
   you're right, this seems like rather a leftover from a copy/paste. removed it



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936853646


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  private Table createTableWithCustomRecords(String tableName) throws IOException {
+    Schema schema =
+        new Schema(
+            required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+    Table tbl =

Review Comment:
   In Iceberg, we use full names, rather than names that leave out vowels. It's typically no harder to type `table` than `tbl` and it's much more readable.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923021336


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public CloseableSample startCloseable() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultCloseableSample(stopwatchRef, defaultTimeUnit, this);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      this.duration = this.duration.plus(amount, toChronoUnit(unit));
+      count.incrementAndGet();
+    }
+  }
+
+  @Override
+  public <T> T record(Supplier<T> supplier) {
+    Sample sample = start();
+    try {
+      return supplier.get();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public <T> T recordCallable(Callable<T> callable) throws Exception {
+    Sample sample = start();
+    try {
+      return callable.call();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public void record(Runnable runnable) {
+    Sample sample = start();
+    try {
+      runnable.run();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("defaultTimeUnit", defaultTimeUnit)
+        .add("count", count)
+        .add("duration", duration)
+        .add("current", stopwatchRef)
+        .toString();
+  }
+
+  private static class DefaultSample implements Sample {
+    private final AtomicReference<Stopwatch> stopwatchRef;
+    private final TimeUnit defaultTimeUnit;
+
+    private DefaultSample(AtomicReference<Stopwatch> stopwatchRef, TimeUnit defaultTimeUnit) {
+      this.stopwatchRef = stopwatchRef;
+      this.defaultTimeUnit = defaultTimeUnit;
+    }
+
+    public void stop(Timer timer) {
+      Preconditions.checkArgument(null != timer, "Timer must be non-null");
+      Stopwatch stopwatch = stopwatchRef.getAndSet(null);
+      Preconditions.checkState(null != stopwatch, "startTimer() was not called");

Review Comment:
   fixed



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r924682755


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final LongAdder count = new LongAdder();
+  private final LongAdder totalTime = new LongAdder();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.longValue();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return Duration.ofNanos(totalTime.longValue());

Review Comment:
   Why is `totalTime` recorded in nanos instead of `defaultTimeUnit`? If we are going to record nanos anyway, why not remove passing the default unit and use nanos everywhere?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922337280


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   I think the problem is that this is tracking state and creating a new object. The object itself tracks state so the caller doesn't need to.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922349612


##########
core/src/main/java/org/apache/iceberg/BaseTableScan.java:
##########
@@ -44,8 +48,21 @@ protected BaseTableScan(TableOperations ops, Table table, Schema schema) {
     this(ops, table, schema, new TableScanContext());
   }
 
+  protected BaseTableScan(TableOperations ops, Table table, Schema schema, ScanReporter scanReporter) {
+    this(ops, table, schema, new TableScanContext(), scanReporter);
+  }
+
   protected BaseTableScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {

Review Comment:
   I think you can go ahead and remove the old methods. These are internal constructors.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922322009


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {
+    /**
+     * Stops the timer and records the total duration up until {@link Timer#start()} was called.
+     */
+    void stop(Timer timer);

Review Comment:
   I think this object should be responsible for tracking its own timer. It's an awkward API to need to pass the right timer back in.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929192438


##########
api/src/main/java/org/apache/iceberg/io/CloseableIterable.java:
##########
@@ -84,6 +113,20 @@ protected boolean shouldKeep(E item) {
     }, iterable);
   }
 
+  static <E> CloseableIterable<E> filter(MetricsContext.Counter<?> counter, CloseableIterable<E> iterable,
+      Predicate<E> pred) {

Review Comment:
   Should there be a version that increments a `skip` counter?
   
   I'd also prefer to add the counter support directly in `FilterIterator` rather than using a predicate. There is no guarantee that the predicate is called only once per value. Putting the increment logic inside `FilterIterator#advance` is better.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923489488


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReporter.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.metrics;
+
+import java.util.Map;
+import org.apache.iceberg.metrics.ScanReport.ScanMetrics;
+
+/**
+ * This interface defines the basic API for a Table Scan Reporter that can be used to collect and report different
+ * metrics during a Table scan. The general usage of the Scan Reporter is as following:
+ * <ul>
+ *   <li>startScan() to indicate that this is a new Scan so that metrics (and other things) can be initialized</li>
+ *   <li>use scanMetrics() to add recorded metrics during a Scan</li>
+ *   <li>reportScan() at the end of the Scan</li>
+ * </ul>
+ */
+public interface ScanReporter {
+
+  /**
+   * Indicates that a new Scan is about to be performed, where implementors should initialize the {@link ScanMetrics},
+   * which can then be used to add recorded metrics to during a Scan.
+   */
+  void startScan();

Review Comment:
   makes sense, updated. The intention was not to make the `ScanReporter` a timer. We're tracking all metrics (including a Timer) inside `ScanMetrics`



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929178393


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;

Review Comment:
   `skipped` sounds reasonable to me!



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926905173


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   regarding `Counter#count()` being `Optional` I talked to @danielcweeks and we eventually concluded that it would probably be best to actually report the count and throw an exception in the Hadoop case. I opened https://github.com/apache/iceberg/pull/5328 for that



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926379231


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")

Review Comment:
   ```
   > Task :iceberg-api:checkstyleMain
   [ant:checkstyle] [ERROR] /home/nastra/Development/workspace/iceberg/api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:141:41: 'tableName' hides a field. [HiddenField]
   [ant:checkstyle] [ERROR] /home/nastra/Development/workspace/iceberg/api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:146:40: 'snapshotId' hides a field. [HiddenField]
   [ant:checkstyle] [ERROR] /home/nastra/Development/workspace/iceberg/api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:151:42: 'filter' hides a field. [HiddenField]
   [ant:checkstyle] [ERROR] /home/nastra/Development/workspace/iceberg/api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:156:42: 'projection' hides a field. [HiddenField]
   [ant:checkstyle] [ERROR] /home/nastra/Development/workspace/iceberg/api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:161:48: 'scanMetrics' hides a field. [HiddenField]
   ```
   
   this is because the parameter names in the Builder methods are named like the field names in the Builder itself



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r934188794


##########
api/src/main/java/org/apache/iceberg/metrics/IntCounter.java:
##########
@@ -55,6 +73,6 @@ public Integer value() {
 
   @Override
   public String toString() {
-    return MoreObjects.toStringHelper(this).add("counter", counter).toString();
+    return value().toString();

Review Comment:
   this is superseded now by https://github.com/apache/iceberg/pull/5386 where the name of the counter is also included



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r931584723


##########
api/src/main/java/org/apache/iceberg/io/IterableUtils.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.io;
+
+import java.util.Iterator;
+import java.util.function.Predicate;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IterableUtils {
+
+  private IterableUtils() {
+  }
+
+  /**
+   * Counts the number of elements in the given {@link Iterable} by incrementing the {@link Counter}
+   * instance for each {@link Iterator#next()} call.
+   *
+   * @param counter The {@link Counter} instance to increment on each {@link Iterator#next()} call.
+   * @param iterable The underlying {@link Iterator} to count
+   * @param <T> The underlying type to be iterated.
+   * @return An {@link Iterable} that increments the given counter on each {@link Iterator#next()}
+   *     call.
+   */
+  public static <T> Iterable<T> count(Counter<?> counter, Iterable<T> iterable) {

Review Comment:
   I think as long as we're introducing a `count` method, having it work with `CloseableIterable` makes the most sense, rather than doing it in a follow-up. I don't think there's a need to introduce the `IterableUtils` class at all, since we could always use the `CloseableIterable` versions.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r932058161


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final ScanMetrics scanMetrics;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      ScanMetrics scanMetrics) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.scanMetrics = scanMetrics;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public ScanMetrics scanMetrics() {
+    return scanMetrics;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("scanMetrics", scanMetrics)
+        .toString();
+  }
+
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String newTableName) {
+      this.tableName = newTableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long newSnapshotId) {
+      this.snapshotId = newSnapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression newFilter) {
+      this.filter = newFilter;
+      return this;
+    }
+
+    public Builder withProjection(Schema newProjection) {
+      this.projection = newProjection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics newScanMetrics) {

Review Comment:
   It feels a bit odd to me to use the `ScanReport.Builder` to retrieve `ScanMetrics`. To me the `ScanReport` feels like something that is final and you'd only want to create at the very end. 
   
   However, I think we can still simplify `ScanReporter` and make it a `FunctionalInterface` that only has `void reportScan(ScanReport scanReport)` and nothing else. 
   `BaseTableScan` doesn't need the `ScanReporter` to init the metrics. It could just do
   ```
   protected ScanReport.ScanMetrics scanMetrics() {
       if (scanMetrics == null) {
         this.scanMetrics = new ScanReport.ScanMetrics(new DefaultMetricsContext());
       }
   
       return scanMetrics;
     }
   ```
   where it can specify whatever metrics context should be used.
   
   Then we just build the `ScanReport` from `ScanMetrics` (and other info) and send it off via `context().scanReporter().reportScan(scanReport)`.
   



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936854467


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);

Review Comment:
   I prefer the tests that add up the file size, rather than just asserting that it's non-zero.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921368933


##########
api/src/main/java/org/apache/iceberg/metrics/LongCounter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.metrics;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A default {@link org.apache.iceberg.metrics.MetricsContext.Counter} implementation that uses a {@link Long} to count
+ * events.
+ */
+class LongCounter implements MetricsContext.Counter<Long> {
+  private final AtomicLong counter;
+
+  LongCounter() {
+    counter = new AtomicLong(0L);

Review Comment:
   When setting instance fields, we use `this.` to show that it is modifying state outside of the local context.



-- 
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-unsubscribe@iceberg.apache.org

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] kbendick commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921526014


##########
api/src/main/java/org/apache/iceberg/io/CloseableIterable.java:
##########
@@ -75,6 +75,31 @@ public CloseableIterator<E> iterator() {
     };
   }
 
+  /**
+   * Will run the given runnable when {@link CloseableIterable#close()} has been called.
+   *
+   * @param iterable             The underlying {@link CloseableIterable} to iterate over
+   * @param onCompletionRunnable The runnable to run after the underlying iterable was closed
+   * @param <E>                  The type of der underlying iterable
+   * @return A new {@link CloseableIterable} where the runnable will be executed
+   * as the final step after {@link CloseableIterable#close()} has been called
+   */
+  static <E> CloseableIterable<E> whenComplete(CloseableIterable<E> iterable, Runnable onCompletionRunnable) {
+    Preconditions.checkNotNull(onCompletionRunnable, "Cannot execute a null Runnable after completion");
+    return new CloseableIterable<E>() {
+      @Override
+      public void close() throws IOException {
+        iterable.close();
+        onCompletionRunnable.run();

Review Comment:
   Does the `onCompletionRunnable` need to happen in a `try-finally` block in case `iterable.close()` throws?
   
   Might be something you want to add a test for.



##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.StringJoiner;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> current = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public void startTimer() {
+    boolean updated = current.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+  }
+
+  @Override
+  public void stopTimer() {
+    Stopwatch stopwatch = current.getAndSet(null);
+    Preconditions.checkState(null != stopwatch, "startTimer() was not called");
+    record(stopwatch.stop().elapsed(defaultTimeUnit), defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      duration = duration.plus(amount, toChronoUnit(unit));
+      count.incrementAndGet();
+    }
+  }
+
+  @Override
+  public <T> T record(Supplier<T> supplier) {
+    startTimer();
+    try {
+      return supplier.get();
+    } finally {
+      stopTimer();
+    }
+  }
+
+  @Override
+  public <T> T recordCallable(Callable<T> callable) throws Exception {
+    startTimer();
+    try {
+      return callable.call();
+    } finally {
+      stopTimer();
+    }
+  }
+
+  @Override
+  public void record(Runnable runnable) {
+    startTimer();
+    try {
+      runnable.run();
+    } finally {
+      stopTimer();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", DefaultTimer.class.getSimpleName() + "[", "]")
+        .add("count=" + count)
+        .add("duration=" + duration)
+        .toString();
+  }

Review Comment:
   Style / Non-blocking: I'm a big fan of using Guava's `MoreObjects.toStringHelper` like seen here:
   
   https://github.com/apache/iceberg/blob/90fe0edf1a671095e587a53adeb31cc84d01fb89/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java#L72-L78
   
   It more or less gives you the same result but is a lot less mental overhead in my opinion. Up to you if you use it or not though.



-- 
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-unsubscribe@iceberg.apache.org

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] danielcweeks commented on a diff in pull request #5268: Initial ScanMetricsContext / ScanReporting support

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r920221226


##########
api/src/main/java/org/apache/iceberg/metrics/ScanMetricsContext.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.metrics;
+
+import java.util.StringJoiner;
+
+public class ScanMetricsContext implements MetricsContext {

Review Comment:
   The original idea behind the MetricsContext interface was to provide a way to delegate to different implementations for creating telemetry primitives (like Counter, Timer, Guage, etc.).
   
   You can imagine implementations like:
   `HadoopMetricsContext` (this exists, but is a wierd implementation because of how it interacts with FileSystem)
   `DefaultMetricsContext` (provides native counters/timers like you've created)
   `MicroMeterMetricsContext` (delegates the creation to a micrometer implementation)
   
   I think this implementation is sort of straddling the ScanReport and the MetricsContext.
   
   You might consider just having:
   
   ```java
   class ScanReport {
     TableIdentifier identifier;
     Schmea schema;
     ...
     
     class Metrics {
       Counter matchingDataFilesCount = metricsContext.counter("matchingDataFilesCount", Integer.class, Unit.COUNT);
       Counter totalDatamanifestsReadCoun = metricsContext.counter("totalDatamanifestsReadCoun", Integer.class, Unit.COUNT)t;
       ...
       Timer totalScanDuration metricsContext.timer("totalScanDuration", Unit.MILLISECOND);
     }
   }
   ```
   
   



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923481609


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalScanDuration() {

Review Comment:
   the only reason to use `Duration` here is because of the flexibility of converting to other time units and out-of-the-box toString() readability



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926037826


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;

Review Comment:
   I think we should change some of these metrics to be more easily understood by people that see them. To do that, I'd start by establishing some consistent terms that we should use. Here's what I'm thinking:
   * Use "manifest" to refer to a manifest file
   * Use "file" to refer to a content file
   * Use "data" and "delete" to specify whether a manifest/file contains data or deletes
   * Use "filtered" to count what was rejected by a filter
   * Use "scanned" to count what is actually read
   * Use "result" to count the data files that are turned into tasks for the scan (alternatively, we could count `fileTasks`?)
   
   I'm trying to avoid using the term "matching" for a couple of reasons. First, it isn't entirely clear what that means when talking about a data file because the filter evaluation results in data files that need to be read, but don't necessarily match the filter. Second, using "filtered" instead gives a clearer picture of what happened during planning, since "matching" isn't always a result. For example, delete files that match the filter might be used or might not match based on sequence number.
   
   Using those terms gives some additional consistency.
   
   * `dataManifests` / `deleteManifests` - the number of data/delete manifests in the snapshot's manifest list
   * `filteredDataManifests` / `filteredDeleteManifests` - the number of data/delete manifests rejected by the filter
   * `scannedDataManifests` / `scannedDeleteManifests` - the number of data/delete manifests that were actually read
   * `totalDataFiles` / `totalDeleteFiles` - the number of data/delete files in manifests that would be scanned (not actually scanned)
   * `filteredDataFiles` / `filteredDeleteFiles` - the number of data/delete files rejected by the filter
   * `indexedDeleteFiles` - the number of delete files that were loaded into the `DeleteFileIndex`
   * `resultDataFiles` - the number of data files that are selected to be read
   
   Does that sound reasonable?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926352244


##########
api/src/main/java/org/apache/iceberg/metrics/LoggingScanReporter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.metrics;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A default {@link ScanReporter} implementation that logs the {@link ScanReport} to the log file.
+ */
+public class LoggingScanReporter implements ScanReporter {
+  private static final Logger LOG = LoggerFactory.getLogger(LoggingScanReporter.class);
+
+  @Override
+  @SuppressWarnings("Slf4jConstantLogMessage")
+  public void reportScan(ScanReport scanReport) {
+    Preconditions.checkArgument(null != scanReport, "ScanReport must be non-null");
+    LOG.info("Ending table scan");
+    LOG.info(scanReport.toString());

Review Comment:
   fixed



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923027006


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   fixed now



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r933640409


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {

Review Comment:
   I think that we need to validate that each counter is being accumulated correctly. That's a lot of work, so we may want to scope this PR down and add just a few high-level counters and then get more specific. That way we can add counters and tests together to make sure we get all the cases.
   
   How about adding just the manifest counters now and we can expand after that? Really whatever subset you have done and want to get tested is a good start. Could also be result counters.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929186975


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReporter.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.metrics;
+
+import org.apache.iceberg.metrics.ScanReport.ScanMetrics;
+
+/**
+ * This interface defines the basic API for a Table Scan Reporter that can be used to collect and report different
+ * metrics during a Table scan.
+ */
+public interface ScanReporter {
+
+  /**
+   * Indicates that a Scan is done by reporting a {@link ScanReport}. A {@link ScanReport} is usually directly derived
+   * from a {@link ScanMetrics} instance.
+   *
+   * @param scanReport The {@link ScanReport} to report.
+   */
+  void reportScan(ScanReport scanReport);
+
+  /**
+   * Creates a new {@link ScanMetrics} instance that can be used during a table scan to add metrics to.
+   *
+   * @return A new {@link ScanMetrics} instance that can be used during a table scan to add metrics to.
+   */
+  ScanMetrics newScanMetrics();

Review Comment:
   I think this should be `newReportBuilder` and the report should function as `ScanMetrics`.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929193196


##########
api/src/main/java/org/apache/iceberg/io/CloseableIterable.java:
##########
@@ -84,6 +113,20 @@ protected boolean shouldKeep(E item) {
     }, iterable);
   }
 
+  static <E> CloseableIterable<E> filter(MetricsContext.Counter<?> counter, CloseableIterable<E> iterable,
+      Predicate<E> pred) {

Review Comment:
   If you want, we can also make a subclass that has callbacks for incrementing, so that we don't need to use NOOP counters when counters aren't supplied.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#issuecomment-1204328588

   Thanks @nastra! I just merged this.


-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r930753325


##########
api/src/main/java/org/apache/iceberg/io/CloseableIterable.java:
##########
@@ -75,6 +76,34 @@ public CloseableIterator<E> iterator() {
     };
   }
 
+  /**
+   * Will run the given runnable when {@link CloseableIterable#close()} has been called.
+   *
+   * @param iterable             The underlying {@link CloseableIterable} to iterate over
+   * @param onCompletionRunnable The runnable to run after the underlying iterable was closed
+   * @param <E>                  The type of der underlying iterable

Review Comment:
   I accidentally leaked german into the code, good catch :)



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r930762661


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int resultDataFiles;
+  private final int totalDataManifests;
+  private final int totalDeleteManifests;
+  private final int scannedDataManifests;
+  private final int scannedDeleteManifests;
+  private final int totalDataFiles;
+  private final int totalDeleteFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeInBytes;
+  private final int indexedDeleteFiles;
+  private final int resultDeleteFiles;
+  private final int skippedDataManifests;
+  private final int skippedDeleteManifests;
+  private final int skippedDataFiles;
+  private final int skippedDeleteFiles;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int resultDataFiles, int totalDataManifests, int totalDeleteManifests, int scannedDataManifests,
+      int scannedDeleteManifests, int totalDataFiles, int totalDeleteFiles, Duration totalPlanningDuration,
+      long totalFileSizeInBytes, int indexedDeleteFiles, int resultDeleteFiles,
+      int skippedDataManifests, int skippedDeleteManifests,
+      int skippedDataFiles, int skippedDeleteFiles) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.resultDataFiles = resultDataFiles;

Review Comment:
   I like that idea, updated the code to reflect that



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929184287


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int resultDataFiles;
+  private final int totalDataManifests;
+  private final int totalDeleteManifests;
+  private final int scannedDataManifests;
+  private final int scannedDeleteManifests;
+  private final int totalDataFiles;
+  private final int totalDeleteFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeInBytes;
+  private final int indexedDeleteFiles;
+  private final int resultDeleteFiles;
+  private final int skippedDataManifests;
+  private final int skippedDeleteManifests;
+  private final int skippedDataFiles;
+  private final int skippedDeleteFiles;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int resultDataFiles, int totalDataManifests, int totalDeleteManifests, int scannedDataManifests,
+      int scannedDeleteManifests, int totalDataFiles, int totalDeleteFiles, Duration totalPlanningDuration,
+      long totalFileSizeInBytes, int indexedDeleteFiles, int resultDeleteFiles,
+      int skippedDataManifests, int skippedDeleteManifests,
+      int skippedDataFiles, int skippedDeleteFiles) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.resultDataFiles = resultDataFiles;
+    this.totalDataManifests = totalDataManifests;
+    this.totalDeleteManifests = totalDeleteManifests;
+    this.scannedDataManifests = scannedDataManifests;
+    this.scannedDeleteManifests = scannedDeleteManifests;
+    this.totalDataFiles = totalDataFiles;
+    this.totalDeleteFiles = totalDeleteFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeInBytes = totalFileSizeInBytes;
+    this.indexedDeleteFiles = indexedDeleteFiles;
+    this.resultDeleteFiles = resultDeleteFiles;
+    this.skippedDataManifests = skippedDataManifests;
+    this.skippedDeleteManifests = skippedDeleteManifests;
+    this.skippedDataFiles = skippedDataFiles;
+    this.skippedDeleteFiles = skippedDeleteFiles;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int resultDataFiles() {
+    return resultDataFiles;
+  }
+
+  public int totalDataManifests() {
+    return totalDataManifests;
+  }
+
+  public int scannedDataManifests() {
+    return scannedDataManifests;
+  }
+
+  public int scannedDeleteManifests() {
+    return scannedDeleteManifests;
+  }
+
+  public int totalDataFiles() {
+    return totalDataFiles;
+  }
+
+  public int totalDeleteFiles() {
+    return totalDeleteFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeInBytes() {
+    return totalFileSizeInBytes;
+  }
+
+  public int indexedDeleteFiles() {
+    return indexedDeleteFiles;
+  }
+
+  public int totalDeleteManifests() {
+    return totalDeleteManifests;
+  }
+
+  public int resultDeleteFiles() {
+    return resultDeleteFiles;
+  }
+
+  public int skippedDataManifests() {
+    return skippedDataManifests;
+  }
+
+  public int skippedDeleteManifests() {
+    return skippedDeleteManifests;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("resultDataFiles", resultDataFiles)
+        .add("resultDeleteFiles", resultDeleteFiles)
+        .add("scannedDataManifests", scannedDataManifests)
+        .add("scannedDeleteManifests", scannedDeleteManifests)
+        .add("totalDataManifests", totalDataManifests)
+        .add("totalDeleteManifests", totalDeleteManifests)
+        .add("totalDataFiles", totalDataFiles)
+        .add("totalDeleteFiles", totalDeleteFiles)
+        .add("totalPlanningDuration", totalPlanningDuration)
+        .add("totalFileSizeInBytes", totalFileSizeInBytes)
+        .add("indexedDeleteFiles", indexedDeleteFiles)
+        .add("skippedDataManifests", skippedDataManifests)
+        .add("skippedDeleteManifests", skippedDeleteManifests)
+        .add("skippedDataFiles", skippedDataFiles)
+        .add("skippedDeleteFiles", skippedDeleteFiles)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.resultDataFiles().value(),
+          scanMetrics.totalDataManifests().value(),
+          scanMetrics.totalDeleteManifests().value(),
+          scanMetrics.scannedDataManifests().value(),
+          scanMetrics.scannedDeleteManifests().value(),
+          scanMetrics.totalDataFiles().value(),
+          scanMetrics.totalDeleteFiles().value(),
+          scanMetrics.totalPlanningDuration().totalDuration(),
+          scanMetrics.totalFileSizeInBytes().value(),
+          scanMetrics.indexedDeleteFiles().value(),
+          scanMetrics.resultDeleteFiles().value(),
+          scanMetrics.skippedDataManifests().value(),
+          scanMetrics.skippedDeleteManifests().value(),
+          scanMetrics.skippedDataFiles().value(),
+          scanMetrics.skippedDeleteFiles.value());
+    }
+  }
+
+  /**
+   * Carries all metrics for a particular scan
+   */
+  public static class ScanMetrics {

Review Comment:
   I think this should be part of `ScanReport` rather than a separate class. I don't see much value in a separate class.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929186137


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")

Review Comment:
   With builders, we typically don't suppress the warning, since it is easy to omit `this.` and not catch that in a review. Instead, it is normal to use argument names like `newTableName` instead of `tableName`.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921368603


##########
api/src/main/java/org/apache/iceberg/metrics/LongCounter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.metrics;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A default {@link org.apache.iceberg.metrics.MetricsContext.Counter} implementation that uses a {@link Long} to count
+ * events.
+ */
+class LongCounter implements MetricsContext.Counter<Long> {
+  private final AtomicLong counter;
+
+  LongCounter() {
+    counter = new AtomicLong(0L);
+  }
+
+  @Override
+  public void increment() {
+    counter.incrementAndGet();
+  }
+
+  @Override
+  public void increment(Long amount) {
+    counter.addAndGet(amount);
+  }
+
+  @Override
+  public Optional<Long> count() {
+    return Optional.of(counter.get());
+  }
+
+  @Override
+  public String toString() {
+    return count().orElse(-1L).toString();

Review Comment:
   This will never be empty, right? Why not just print the value of counter directly?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936160897


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    TableScan tableScan = table.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();
+    TableScan tableScan = tbl.newScan();
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-with-deletes");
+    assertThat(scanReport.snapshotId()).isEqualTo(2L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isEqualTo(30L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(20L);
+  }
+
+  private DataFile writeParquetFile(Table tbl, List<GenericRecord> records) throws IOException {
+    File parquetFile = temp.newFile();
+    assertTrue(parquetFile.delete());
+    FileAppender<GenericRecord> appender =
+        Parquet.write(Files.localOutput(parquetFile))
+            .schema(tbl.schema())
+            .createWriterFunc(GenericParquetWriter::buildWriter)
+            .build();
+    try {
+      appender.addAll(records);
+    } finally {
+      appender.close();
+    }
+
+    PartitionKey partitionKey = new PartitionKey(tbl.spec(), tbl.schema());
+    partitionKey.partition(records.get(0));

Review Comment:
   The table doesn't have any partition fields, so this isn't quite right.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922321453


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {
+    /**
+     * Stops the timer and records the total duration up until {@link Timer#start()} was called.
+     */
+    void stop(Timer timer);
+
+    Sample NOOP = timer -> { };
+  }
+
+  interface CloseableSample extends AutoCloseable {

Review Comment:
   What is the purpose of having this as well as `Sample`? If they do the same thing with different ways of stopping the timed period, then can't one interface provide both `close` and `stop`?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921884393


##########
api/src/main/java/org/apache/iceberg/metrics/LongCounter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.metrics;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A default {@link org.apache.iceberg.metrics.MetricsContext.Counter} implementation that uses a {@link Long} to count
+ * events.
+ */
+class LongCounter implements MetricsContext.Counter<Long> {
+  private final AtomicLong counter;
+
+  LongCounter() {
+    counter = new AtomicLong(0L);
+  }
+
+  @Override
+  public void increment() {
+    counter.incrementAndGet();
+  }
+
+  @Override
+  public void increment(Long amount) {
+    counter.addAndGet(amount);
+  }
+
+  @Override
+  public Optional<Long> count() {
+    return Optional.of(counter.get());
+  }
+
+  @Override
+  public String toString() {
+    return count().orElse(-1L).toString();

Review Comment:
   makes sense, updated



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921886698


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();

Review Comment:
   good point, I updated this to just start() / stop()



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921903395


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.StringJoiner;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> current = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public void startTimer() {
+    boolean updated = current.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");

Review Comment:
   you're absolutely right about this. That's also how Micrometer handles timing state. I'll update accordingly



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922312439


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);

Review Comment:
   I agree with using `record` for direct additions, but I don't think I would make the names the same across something that is timed by the timer and a time that is provided by the caller.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922332725


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalScanDuration() {
+    return totalScanDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalScanDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFilesCount().count().orElse(-1),
+          scanMetrics.matchingDataManifestsCounter().count().orElse(-1),
+          scanMetrics.totalDataManifestsReadCount().count().orElse(-1),
+          scanMetrics.addedDataFiles().count().orElse(-1),
+          scanMetrics.deletedDataFiles().count().orElse(-1),
+          scanMetrics.totalScanDuration().totalDuration(),
+          scanMetrics.totalFileSizeInBytes().count().orElse(-1L));
+    }
+  }
+
+  /**
+   * Carries all metrics for a particular scan
+   */
+  public static class ScanMetrics {
+    public static final ScanMetrics NOOP = new ScanMetrics(MetricsContext.nullMetrics());
+    private final Counter<Integer> matchingDataFilesCount;
+    private final Counter<Integer> totalDataManifestsReadCount;
+    private final Counter<Integer> matchingDataManifestsCounter;
+    private final Counter<Integer> addedDataFiles;
+    private final Counter<Integer> deletedDataFiles;
+    private final Counter<Long> totalFileSizeInBytes;
+    private final Timer totalScanDuration;
+
+    public ScanMetrics(MetricsContext metricsContext) {
+      Preconditions.checkArgument(null != metricsContext, "MetricsContext must be non-null");
+      this.matchingDataFilesCount =
+          metricsContext.counter("matchingDataFilesCount", Integer.class, MetricsContext.Unit.COUNT);
+      this.totalDataManifestsReadCount =
+          metricsContext.counter("totalDataManifestsReadCount", Integer.class, MetricsContext.Unit.COUNT);
+      this.matchingDataManifestsCounter =
+          metricsContext.counter("matchingDataManifestsCounter", Integer.class, MetricsContext.Unit.COUNT);
+      this.addedDataFiles = metricsContext.counter("addedDataFiles", Integer.class, MetricsContext.Unit.COUNT);
+      this.deletedDataFiles = metricsContext.counter("deletedDataFiles", Integer.class, MetricsContext.Unit.COUNT);
+      this.totalFileSizeInBytes = metricsContext.counter("totalFileSizeInBytes", Long.class, MetricsContext.Unit.BYTES);
+      this.totalScanDuration = metricsContext.timer("totalScanDuration", TimeUnit.NANOSECONDS);
+    }
+
+    public Counter<Integer> matchingDataFilesCount() {

Review Comment:
   I don't think this needs the `Count` suffix. It's fairly clear that this is going to be a counter.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921988864


##########
api/src/main/java/org/apache/iceberg/io/CloseableIterable.java:
##########
@@ -75,6 +75,31 @@ public CloseableIterator<E> iterator() {
     };
   }
 
+  /**
+   * Will run the given runnable when {@link CloseableIterable#close()} has been called.
+   *
+   * @param iterable             The underlying {@link CloseableIterable} to iterate over
+   * @param onCompletionRunnable The runnable to run after the underlying iterable was closed
+   * @param <E>                  The type of der underlying iterable
+   * @return A new {@link CloseableIterable} where the runnable will be executed
+   * as the final step after {@link CloseableIterable#close()} has been called
+   */
+  static <E> CloseableIterable<E> whenComplete(CloseableIterable<E> iterable, Runnable onCompletionRunnable) {
+    Preconditions.checkNotNull(onCompletionRunnable, "Cannot execute a null Runnable after completion");
+    return new CloseableIterable<E>() {
+      @Override
+      public void close() throws IOException {
+        iterable.close();
+        onCompletionRunnable.run();

Review Comment:
   good catch, that makes definitely sense. Updated and added a test



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922043533


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   Given that we're tracking state, using `AutoCloseable` (bf2178c818508152dce7a3b910a998a5fba9d2b5) makes the API less readable imo. I've done it in a separate commit, so we can still decide whether we want to have it or not



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922323024


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {
+    /**
+     * Stops the timer and records the total duration up until {@link Timer#start()} was called.
+     */
+    void stop(Timer timer);
+
+    Sample NOOP = timer -> { };
+  }
+
+  interface CloseableSample extends AutoCloseable {
+    @Override
+    void close();
+    CloseableSample NOOP = () -> { };
+  }
+
+  Timer NOOP = new Timer() {
+    @Override
+    public Sample start() {
+      return Sample.NOOP;
+    }
+
+    @Override
+    public CloseableSample startCloseable() {
+      return CloseableSample.NOOP;
+    }
+
+    @Override
+    public long count() {
+      return 0;
+    }
+
+    @Override
+    public Duration totalDuration() {
+      return Duration.ZERO;
+    }
+
+    @Override
+    public void record(long amount, TimeUnit unit) {
+    }
+
+    @Override
+    public void record(Runnable runnable) {

Review Comment:
   As I mentioned above, I think that timing the execution of something is a different operation than recording an interval that was timed externally.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922320815


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {

Review Comment:
   When I think of sampling in metrics, I think of randomly deciding whether to time or use a no-op, to avoid overhead in tight loops. It looks like the intent here is to make this something that reports when stopped, so a different name is probably more clear (I had no idea why this had sampling).
   
   How about calling this `Timed` or something?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922328215


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {

Review Comment:
   What is `deletedDataFiles` reporting?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936858257


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  private Table createTableWithCustomRecords(String tableName) throws IOException {
+    Schema schema =
+        new Schema(
+            required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            tableName,
+            schema,
+            PartitionSpec.builderFor(schema).build(),
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(tbl, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(tbl, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(tbl, Arrays.asList(record4));

Review Comment:
   Can you add some whitespace here to make this more readable?
   
   Maybe break up record creation from writes. Also, if you use Iceberg's `GenericRecord`, it's usually easier to create records. You just create one from the schema and call `copy(Map<String, Object>)` to make a new one with data values.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936852825


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  private Table createTableWithCustomRecords(String tableName) throws IOException {
+    Schema schema =
+        new Schema(
+            required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            tableName,
+            schema,
+            PartitionSpec.builderFor(schema).build(),
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(tbl, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(tbl, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(tbl, Arrays.asList(record4));
+    tbl.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    tbl.refresh();

Review Comment:
   No need to call `refresh`.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936430695


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    TableScan tableScan = table.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();

Review Comment:
   ah good point, I wasn't sure initially how to test that but after looking at some other tests I was able to write up a test that has multiple data manifests and data files, where we can verify how many data manifests we scan & skip



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923167463


##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -59,6 +76,10 @@ protected TableScanContext context() {
     return context;
   }
 
+  protected ScanReporter scanReporter() {

Review Comment:
   subclasses need access to this and checkstyle is complaining that this needs to be exposed via a method for subclasses



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923443028


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -234,7 +255,13 @@ private <T> Iterable<CloseableIterable<T>> entries(
           manifest -> manifest.hasAddedFiles() || manifest.hasDeletedFiles());
     }
 
-    matchingManifests = Iterables.filter(matchingManifests, manifestPredicate::test);
+    matchingManifests = Iterables.filter(matchingManifests, manifest -> {

Review Comment:
   that would obviously also work and I'm completely open to suggestions. The only downside of this approach is that we're creating a new iterator



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923485868


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalScanDuration() {
+    return totalScanDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalScanDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFilesCount().count().orElse(-1),
+          scanMetrics.matchingDataManifestsCounter().count().orElse(-1),
+          scanMetrics.totalDataManifestsReadCount().count().orElse(-1),
+          scanMetrics.addedDataFiles().count().orElse(-1),
+          scanMetrics.deletedDataFiles().count().orElse(-1),
+          scanMetrics.totalScanDuration().totalDuration(),
+          scanMetrics.totalFileSizeInBytes().count().orElse(-1L));
+    }
+  }
+
+  /**
+   * Carries all metrics for a particular scan
+   */
+  public static class ScanMetrics {
+    public static final ScanMetrics NOOP = new ScanMetrics(MetricsContext.nullMetrics());
+    private final Counter<Integer> matchingDataFilesCount;
+    private final Counter<Integer> totalDataManifestsReadCount;
+    private final Counter<Integer> matchingDataManifestsCounter;
+    private final Counter<Integer> addedDataFiles;
+    private final Counter<Integer> deletedDataFiles;
+    private final Counter<Long> totalFileSizeInBytes;
+    private final Timer totalScanDuration;
+
+    public ScanMetrics(MetricsContext metricsContext) {
+      Preconditions.checkArgument(null != metricsContext, "MetricsContext must be non-null");
+      this.matchingDataFilesCount =
+          metricsContext.counter("matchingDataFilesCount", Integer.class, MetricsContext.Unit.COUNT);
+      this.totalDataManifestsReadCount =
+          metricsContext.counter("totalDataManifestsReadCount", Integer.class, MetricsContext.Unit.COUNT);
+      this.matchingDataManifestsCounter =
+          metricsContext.counter("matchingDataManifestsCounter", Integer.class, MetricsContext.Unit.COUNT);
+      this.addedDataFiles = metricsContext.counter("addedDataFiles", Integer.class, MetricsContext.Unit.COUNT);
+      this.deletedDataFiles = metricsContext.counter("deletedDataFiles", Integer.class, MetricsContext.Unit.COUNT);
+      this.totalFileSizeInBytes = metricsContext.counter("totalFileSizeInBytes", Long.class, MetricsContext.Unit.BYTES);
+      this.totalScanDuration = metricsContext.timer("totalScanDuration", TimeUnit.NANOSECONDS);
+    }
+
+    public Counter<Integer> matchingDataFilesCount() {

Review Comment:
   updated



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929583872


##########
api/src/main/java/org/apache/iceberg/metrics/MetricsContext.java:
##########
@@ -126,15 +126,14 @@ public Timer timer(String name, TimeUnit unit) {
 
       @Override
       public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
-        return new Counter<T>() {
-          @Override
-          public void increment() {
-          }
-
-          @Override
-          public void increment(T amount) {
-          }
-        };
+        if (Integer.class.equals(type)) {
+          return (Counter<T>) IntCounter.NOOP;
+        }
+
+        if (Long.class.equals(type)) {
+          return (Counter<T>) LongCounter.NOOP;
+        }
+        throw new IllegalArgumentException(String.format("Counter for type %s is not supported", type.getName()));

Review Comment:
   fixed



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r924680402


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final LongAdder count = new LongAdder();
+  private final LongAdder totalTime = new LongAdder();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.longValue();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return Duration.ofNanos(totalTime.longValue());
+  }
+
+  @Override
+  public Timed start() {
+    return new DefaultTimed(this, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    Preconditions.checkArgument(amount >= 0, "Cannot record %s %s: must be >= 0", amount, unit);
+    this.totalTime.add(TimeUnit.NANOSECONDS.convert(amount, unit));
+    this.count.increment();
+  }
+
+  @Override
+  public <T> T time(Supplier<T> supplier) {
+    Timed timed = start();
+    try {
+      return supplier.get();
+    } finally {
+      timed.stop();
+    }
+  }
+
+  @Override
+  public <T> T timeCallable(Callable<T> callable) throws Exception {
+    Timed timed = start();
+    try {
+      return callable.call();
+    } finally {
+      timed.stop();
+    }
+  }
+
+  @Override
+  public void time(Runnable runnable) {
+    Timed timed = start();
+    try {
+      runnable.run();
+    } finally {
+      timed.stop();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("defaultTimeUnit", defaultTimeUnit)
+        .add("count", count)
+        .add("duration", totalDuration())
+        .toString();
+  }
+
+  private static class DefaultTimed implements Timed {
+    private final Timer timer;
+    private final TimeUnit defaultTimeUnit;
+    private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+    private DefaultTimed(Timer timer, TimeUnit defaultTimeUnit) {
+      this.timer = timer;
+      this.defaultTimeUnit = defaultTimeUnit;
+      stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    }
+
+    @Override
+    public void stop() {
+      Stopwatch stopwatch = stopwatchRef.getAndSet(null);
+      Preconditions.checkState(null != stopwatch, "startTimer() was not called");

Review Comment:
   There is no `startTimer` method anymore. If `stopwatch` is null, that indicates that `stop` was called more than once.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r924682755


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final LongAdder count = new LongAdder();
+  private final LongAdder totalTime = new LongAdder();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.longValue();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return Duration.ofNanos(totalTime.longValue());

Review Comment:
   Why is `totalTime` recorded in nanos instead of `defaultTimeUnit`?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936161284


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    TableScan tableScan = table.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();

Review Comment:
   You could use this setup to easily test the manifest file counting. I think these are partitioned files, so you'd just have to write them to separate manifests.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929184697


##########
api/src/main/java/org/apache/iceberg/metrics/MetricsContext.java:
##########
@@ -126,15 +126,14 @@ public Timer timer(String name, TimeUnit unit) {
 
       @Override
       public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
-        return new Counter<T>() {
-          @Override
-          public void increment() {
-          }
-
-          @Override
-          public void increment(T amount) {
-          }
-        };
+        if (Integer.class.equals(type)) {
+          return (Counter<T>) IntCounter.NOOP;
+        }
+
+        if (Long.class.equals(type)) {
+          return (Counter<T>) LongCounter.NOOP;
+        }
+        throw new IllegalArgumentException(String.format("Counter for type %s is not supported", type.getName()));

Review Comment:
   Style: there should be a newline between a control flow block and the follow statement.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922352408


##########
core/src/main/java/org/apache/iceberg/CatalogUtil.java:
##########
@@ -346,4 +347,42 @@ public static void configureHadoopConf(Object maybeConfigurable, Object conf) {
 
     setConf.invoke(conf);
   }
+
+  /**
+   * Load a custom {@link ScanReporter} implementation.
+   * <p>
+   * The implementation must have a no-arg constructor. {@link ScanReporter#initialize(Map properties)} is called to
+   * complete the initialization.
+   *
+   * @param impl       full class name of a custom {@link ScanReporter} implementation
+   * @param properties used to initialize the {@link ScanReporter} implementation
+   * @return An initialized {@link ScanReporter}.
+   * @throws IllegalArgumentException if class path not found or right constructor not found or the loaded class cannot
+   *                                  be cast to the given interface type
+   */
+  public static ScanReporter loadScanReporter(String impl, Map<String, String> properties) {

Review Comment:
   Since this is passed in from the table, we don't need to support dynamic loading. If there's a use case, we can add it later.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922348325


##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -59,6 +76,10 @@ protected TableScanContext context() {
     return context;
   }
 
+  protected ScanReporter scanReporter() {

Review Comment:
   Does this need to be exposed?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922317403


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultMetricsContext.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A default {@link MetricsContext} implementation that uses native Java counters/timers.
+ */
+public class DefaultMetricsContext implements MetricsContext {
+
+  @Override
+  public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
+    if (Integer.class.equals(type)) {
+      return (Counter<T>) new IntCounter();
+    }
+    if (Long.class.equals(type)) {

Review Comment:
   Style: we add an empty newline between control flow blocks and the following statement.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922368487


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -163,20 +171,26 @@ public CloseableIterable<FileScanTask> planFiles() {
     });
 
     DeleteFileIndex deleteFiles = deleteIndexBuilder.build();
-
     boolean dropStats = ManifestReader.dropStats(dataFilter, columns);
     if (!deleteFiles.isEmpty()) {
       select(ManifestReader.withStatsColumns(columns));
     }
 
     Iterable<CloseableIterable<FileScanTask>> tasks = entries((manifest, entries) -> {
+      scanMetrics.addedDataFiles().increment(manifest.addedFilesCount());
+      scanMetrics.deletedDataFiles().increment(manifest.deletedFilesCount());

Review Comment:
   I think I see what you're doing with these counters now. These end up tracking the total number of data or delete files in manifests that are scanned, right?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922367154


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -215,8 +229,15 @@ private <T> Iterable<CloseableIterable<T>> entries(
       evaluator = null;
     }
 
+    if (evalCache == null) {
+      scanMetrics.totalDataManifestsReadCount().increment(dataManifests.size());
+    }

Review Comment:
   Style: missing newline.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922371347


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -163,20 +171,26 @@ public CloseableIterable<FileScanTask> planFiles() {
     });
 
     DeleteFileIndex deleteFiles = deleteIndexBuilder.build();
-

Review Comment:
   Nit: unnecessary whitespace change.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921892961


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);

Review Comment:
   I was broadly following how Micrometer defines their [Timer](https://www.javadoc.io/doc/io.micrometer/micrometer-core/latest/io/micrometer/core/instrument/Timer.html) interface and I had the exact same thought as you mentioned here. 
   That being said, `time()` makes also sense to me as a verb for measuring a Runnable/Callable/Supplier, but I think `record(long amount, TimeUnit unit)` / `record(Duration duration)` reads nicer than `time(long amount, TimeUnit unit)` / `time(Duration duration)` and in order to keep method names aligned I decided to go with `record()` across the board. 
   



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r925711949


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -234,7 +255,13 @@ private <T> Iterable<CloseableIterable<T>> entries(
           manifest -> manifest.hasAddedFiles() || manifest.hasDeletedFiles());
     }
 
-    matchingManifests = Iterables.filter(matchingManifests, manifestPredicate::test);
+    matchingManifests = Iterables.filter(matchingManifests, manifest -> {

Review Comment:
   added that functionality



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r925859764


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   Following up on [this comment](https://github.com/apache/iceberg/pull/5268#discussion_r922334117), I don't think that this should produce -1.
   
   The reason why `count` returns an `Optional` is that implementations, like Hadoop counters, may not be able to return the final value. Those counters aren't appropriate here, because we need to get the value to build a scan report.
   
   There are a couple options to fix. One is to log an error and not send the scan report, so this method would return null. I'm not a big fan of that, or of throwing an exception because metrics collection should not fail scans.
   
   The fix that I think makes the most sense is to let the `ScanReporter` implementation handle all of this internally. There's not really a need to have separate classes for `ScanMetrics` and `ScanReport`, when the report just tries to get the final metrics values. Separating the two classes creates a situation where the Iceberg library needs to interpret the `ScanMetrics` implementation that is provided by the `ScanReporter`. Instead, I think it's simpler just to pass the `ScanMetrics` (which were created by the `ScanReporter`) back as the final report.
   
   I think removing `ScanReport` and having just `ScanMetrics` is a cleaner in general as well. I don't see a clear purpose for a separate `ScanReport`, and a simpler API is generally better.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926908639


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;

Review Comment:
   I think instead of `filteredX` we might rather want to use `skippedX`. Depending on the context, `filtered` could mean something is included or excluded but `skipped` should be unambiguous.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r931584723


##########
api/src/main/java/org/apache/iceberg/io/IterableUtils.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.io;
+
+import java.util.Iterator;
+import java.util.function.Predicate;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IterableUtils {
+
+  private IterableUtils() {
+  }
+
+  /**
+   * Counts the number of elements in the given {@link Iterable} by incrementing the {@link Counter}
+   * instance for each {@link Iterator#next()} call.
+   *
+   * @param counter The {@link Counter} instance to increment on each {@link Iterator#next()} call.
+   * @param iterable The underlying {@link Iterator} to count
+   * @param <T> The underlying type to be iterated.
+   * @return An {@link Iterable} that increments the given counter on each {@link Iterator#next()}
+   *     call.
+   */
+  public static <T> Iterable<T> count(Counter<?> counter, Iterable<T> iterable) {

Review Comment:
   I think as long as we're introducing a `count` method, having it work with `CloseableIterable` makes the most sense, rather than doing it in a follow-up.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r932060125


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   @danielcweeks do you mean https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/metrics/MetricsContext.java#L105-L107? I think we should be good here as the timer creation goes through the `MetricsContext`



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r933625359


##########
api/src/main/java/org/apache/iceberg/metrics/IntCounter.java:
##########
@@ -55,6 +73,6 @@ public Integer value() {
 
   @Override
   public String toString() {
-    return MoreObjects.toStringHelper(this).add("counter", counter).toString();
+    return value().toString();

Review Comment:
   Minor: I think it would be strange to print this and see just the value, but it's probably okay.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r934210627


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -184,12 +192,14 @@ public <T extends ScanTask> CloseableIterable<T> plan(CreateTasksFunction<T> cre
                 specId -> {
                   PartitionSpec spec = specsById.get(specId);
                   ResidualEvaluator residuals = residualCache.get(specId);
-                  return new TaskContext(spec, deleteFiles, residuals, dropStats);
+                  return new TaskContext(spec, deleteFiles, residuals, dropStats, scanMetrics);
                 });
 
     Iterable<CloseableIterable<T>> tasks =
         entries(
             (manifest, entries) -> {
+              scanMetrics.totalDataFiles().increment(manifest.addedFilesCount());
+              scanMetrics.totalDeleteFiles().increment(manifest.deletedFilesCount());

Review Comment:
   ah that makes sense, thanks for double checking this. I added option 2 and I'm counting total data/delete files in the `ManifestReader`



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936160673


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    TableScan tableScan = table.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-x");
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  @Test
+  public void deleteScanning() throws IOException {
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            "scan-planning-with-deletes",
+            SCHEMA,
+            SPEC,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+
+    tbl.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit();
+    tbl.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit();
+    TableScan tableScan = tbl.newScan();
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo("scan-planning-with-deletes");
+    assertThat(scanReport.snapshotId()).isEqualTo(2L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);

Review Comment:
   I think we are going to need a test for skipped data manifests. I think you could probably achieve that with the current tests by splitting the write across 2 manifests and using a truncate(50) partition transform or just duplicating the test and truncating id instead of x. You may have to adjust values in the files a bit, but it's pretty close.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936909668


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  private Table createTableWithCustomRecords(String tableName) throws IOException {
+    Schema schema =
+        new Schema(
+            required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+    Table tbl =

Review Comment:
   I probably accidentally named it this way to not interfere with the `table` in the super class. should be fixed now



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936907140


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.greaterThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(2);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be 1 file
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "30")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // all files
+    try (CloseableIterable<FileScanTask> fileScanTasks =
+        tableScan.filter(Expressions.lessThan("x", "52")).planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+  }
+
+  private Table createTableWithCustomRecords(String tableName) throws IOException {
+    Schema schema =
+        new Schema(
+            required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+    Table tbl =
+        TestTables.create(
+            tableDir,
+            tableName,
+            schema,
+            PartitionSpec.builderFor(schema).build(),
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(tbl, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(tbl, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(tbl, Arrays.asList(record4));

Review Comment:
   thanks for the tip. updated. I left record creation and writes together for now, but I'm planning to re-visit and improving it later (when adding & testing new metrics)



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue merged pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #5268:
URL: https://github.com/apache/iceberg/pull/5268


-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921907936


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;

Review Comment:
   I think it would also make sense to have something like a `TimedRunnable` / `TimedSupplier`



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921354439


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);

Review Comment:
   What about using `time` as the verb? I think `record` makes sense for adding intervals timed in some other way, but I would expect `time` to be used when passing a `Runnable` or `Callable`.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922331382


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {

Review Comment:
   I'm assuming that this is distinct from `matchingDataManifests` because this is the number that were actually read if the scan planning iterable was closed early?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921887165


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   good point, that makes perfect sense to stop the timer when close is called



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921884989


##########
api/src/main/java/org/apache/iceberg/metrics/LongCounter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.metrics;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A default {@link org.apache.iceberg.metrics.MetricsContext.Counter} implementation that uses a {@link Long} to count
+ * events.
+ */
+class LongCounter implements MetricsContext.Counter<Long> {
+  private final AtomicLong counter;
+
+  LongCounter() {
+    counter = new AtomicLong(0L);

Review Comment:
   makes sense, updated



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929608602


##########
api/src/main/java/org/apache/iceberg/io/CloseableIterable.java:
##########
@@ -84,6 +113,20 @@ protected boolean shouldKeep(E item) {
     }, iterable);
   }
 
+  static <E> CloseableIterable<E> filter(MetricsContext.Counter<?> counter, CloseableIterable<E> iterable,
+      Predicate<E> pred) {

Review Comment:
   this version is actually incrementing a `skip` counter. I just forgot adding some javadocs to it.
   
   Based on the current contract of [FilterIterator](https://github.com/apache/iceberg/blob/bbcc2fbf6db6f5386fa9e6d169a4cc19d7883ef2/api/src/main/java/org/apache/iceberg/io/FilterIterator.java#L67), it seems that `shouldKeep` is only called once. I tried to keep the changes for counting skipped elements to a minimum, so is this somethind we'd want to do in a follow-up PR?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929610469


##########
api/src/main/java/org/apache/iceberg/io/IterableUtils.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.io;
+
+import java.util.Iterator;
+import java.util.function.Predicate;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public class IterableUtils {
+
+  private IterableUtils() {
+  }
+
+  /**
+   * Counts the number of elements in the given {@link Iterable} by incrementing the {@link Counter}
+   * instance for each {@link Iterator#next()} call.
+   *
+   * @param counter The {@link Counter} instance to increment on each {@link Iterator#next()} call.
+   * @param iterable The underlying {@link Iterator} to count
+   * @param <T> The underlying type to be iterated.
+   * @return An {@link Iterable} that increments the given counter on each {@link Iterator#next()}
+   *     call.
+   */
+  public static <T> Iterable<T> count(Counter<?> counter, Iterable<T> iterable) {

Review Comment:
   Is this somethind we'd want to do as a follow-up? As it would require to change a few places and I didn't want to pollute this PR with semi-related 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923482774


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {

Review Comment:
   yes, the idea is to add more stuff eventually to this class (depending on the things we'd like to track)



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923439422


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -163,20 +171,26 @@ public CloseableIterable<FileScanTask> planFiles() {
     });
 
     DeleteFileIndex deleteFiles = deleteIndexBuilder.build();
-
     boolean dropStats = ManifestReader.dropStats(dataFilter, columns);
     if (!deleteFiles.isEmpty()) {
       select(ManifestReader.withStatsColumns(columns));
     }
 
     Iterable<CloseableIterable<FileScanTask>> tasks = entries((manifest, entries) -> {
+      scanMetrics.addedDataFiles().increment(manifest.addedFilesCount());
+      scanMetrics.deletedDataFiles().increment(manifest.deletedFilesCount());

Review Comment:
   yes correct. The naming is probably bad in `ScanMetrics`, so I'm completely open to suggestions on naming and also on what metrics to track and where to retrieve them from. The initial idea was that there are just **some metrics** gathered and reported



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r924123676


##########
core/src/main/java/org/apache/iceberg/BaseTableScan.java:
##########
@@ -116,9 +133,20 @@ public CloseableIterable<FileScanTask> planFiles() {
           ExpressionUtil.toSanitizedString(filter()));
 
       Listeners.notifyAll(new ScanEvent(table().name(), snapshot.snapshotId(), filter(), schema()));
-
-      return doPlanFiles();
-
+      scanReporter().startScan();
+      Timer.Sample sample = scanReporter().scanMetrics().totalScanDuration().start();

Review Comment:
   I double-checked and the old implementation was behaving correctly, because a new `ScanMetrics` instance was created when `ScanReporter#startScan()` was called.
   However, I updated the `ScanReporter` implementation to how you suggested, as that is cleaner



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r924685319


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final LongAdder count = new LongAdder();
+  private final LongAdder totalTime = new LongAdder();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.longValue();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return Duration.ofNanos(totalTime.longValue());
+  }
+
+  @Override
+  public Timed start() {
+    return new DefaultTimed(this, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    Preconditions.checkArgument(amount >= 0, "Cannot record %s %s: must be >= 0", amount, unit);
+    this.totalTime.add(TimeUnit.NANOSECONDS.convert(amount, unit));
+    this.count.increment();
+  }
+
+  @Override
+  public <T> T time(Supplier<T> supplier) {
+    Timed timed = start();
+    try {
+      return supplier.get();
+    } finally {
+      timed.stop();
+    }
+  }
+
+  @Override
+  public <T> T timeCallable(Callable<T> callable) throws Exception {
+    Timed timed = start();
+    try {
+      return callable.call();
+    } finally {
+      timed.stop();
+    }
+  }
+
+  @Override
+  public void time(Runnable runnable) {
+    Timed timed = start();
+    try {
+      runnable.run();
+    } finally {
+      timed.stop();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("defaultTimeUnit", defaultTimeUnit)
+        .add("count", count)
+        .add("duration", totalDuration())
+        .toString();
+  }
+
+  private static class DefaultTimed implements Timed {
+    private final Timer timer;
+    private final TimeUnit defaultTimeUnit;
+    private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+    private DefaultTimed(Timer timer, TimeUnit defaultTimeUnit) {
+      this.timer = timer;
+      this.defaultTimeUnit = defaultTimeUnit;
+      stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    }
+
+    @Override
+    public void stop() {
+      Stopwatch stopwatch = stopwatchRef.getAndSet(null);
+      Preconditions.checkState(null != stopwatch, "startTimer() was not called");

Review Comment:
   ah yeah I need to adjust the error message to say `stop() called multiple times`



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r924687023


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final LongAdder count = new LongAdder();
+  private final LongAdder totalTime = new LongAdder();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.longValue();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return Duration.ofNanos(totalTime.longValue());

Review Comment:
   oversight after cleaning up the implementation :( will push a fix



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923207134


##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -59,6 +76,10 @@ protected TableScanContext context() {
     return context;
   }
 
+  protected ScanReporter scanReporter() {

Review Comment:
   moved the `ScanReporter` into the `TableScanContext`, so this change isn't required anymore



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r925994923


##########
api/src/main/java/org/apache/iceberg/metrics/LoggingScanReporter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.metrics;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A default {@link ScanReporter} implementation that logs the {@link ScanReport} to the log file.
+ */
+public class LoggingScanReporter implements ScanReporter {
+  private static final Logger LOG = LoggerFactory.getLogger(LoggingScanReporter.class);
+
+  @Override
+  @SuppressWarnings("Slf4jConstantLogMessage")
+  public void reportScan(ScanReport scanReport) {
+    Preconditions.checkArgument(null != scanReport, "ScanReport must be non-null");

Review Comment:
   We typically form error messages by telling you what's wrong, not what the requirement is. That would be `"Invalid scan report: null"`
   
   Another convention that we use is that we avoid class and method names in error messages. The person reading the error rarely has the context to know what the variable is called, and variable names tend to get out of sync with strings (as you could see from the error message about `startTimer()` not getting called).



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936855345


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files

Review Comment:
   I don't think there's a need for all 3 cases. How about testing all 3 files and just 1 file? I don't think that there is much of a difference between the 2 and 1 cases.



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r934275343


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final Schema schema =
+      new Schema(
+          required(1, "id", Types.IntegerType.get()), required(2, "x", Types.StringType.get()));
+
+  PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build();
+
+  private Table table;
+  private TestScanReporter reporter;
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Before
+  @Override
+  public void setupTable() throws Exception {
+    super.setupTable();
+    reporter = new TestScanReporter();
+    table =
+        TestTables.create(
+            tableDir,
+            "scan-planning-x",
+            schema,
+            partitionSpec,
+            SortOrder.unsorted(),
+            formatVersion,
+            reporter);
+    GenericRecord record = GenericRecord.create(schema);
+    record.setField("id", 1);
+    record.setField("x", "23");
+    GenericRecord record2 = GenericRecord.create(schema);
+    record2.setField("id", 2);
+    record2.setField("x", "30");
+    GenericRecord record3 = GenericRecord.create(schema);
+    record3.setField("id", 3);
+    record3.setField("x", "45");
+    GenericRecord record4 = GenericRecord.create(schema);
+    record4.setField("id", 3);
+    record4.setField("x", "51");
+    DataFile dataFile = writeParquetFile(table, Arrays.asList(record, record3));
+    DataFile dataFile2 = writeParquetFile(table, Arrays.asList(record2));
+    DataFile dataFile3 = writeParquetFile(table, Arrays.asList(record4));
+    table.newFastAppend().appendFile(dataFile).appendFile(dataFile2).appendFile(dataFile3).commit();
+    table.refresh();
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {

Review Comment:
   that sounds good to me. Let me remove some counters and push a new version



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r933629987


##########
core/src/main/java/org/apache/iceberg/DeleteFileIndex.java:
##########
@@ -502,6 +509,9 @@ DeleteFileIndex build() {
         }
       }
 
+      if (null != globalDeletes) {
+        scanMetrics.indexedDeleteFiles().increment(globalDeletes.length);
+      }

Review Comment:
   I don't think this is correct. It should be the number of delete files that were indexed for all partitions, not just the root partition. I think you should count the entries that are processed in the first loop, or add up the number of existing and added entries in the matching delete files as they are read (near `readDeleteManifest`).



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r925859764


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   Following up on this comment, I 



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r926497119


##########
api/src/main/java/org/apache/iceberg/metrics/LoggingScanReporter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.metrics;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A default {@link ScanReporter} implementation that logs the {@link ScanReport} to the log file.
+ */
+public class LoggingScanReporter implements ScanReporter {
+  private static final Logger LOG = LoggerFactory.getLogger(LoggingScanReporter.class);
+
+  @Override
+  @SuppressWarnings("Slf4jConstantLogMessage")
+  public void reportScan(ScanReport scanReport) {
+    Preconditions.checkArgument(null != scanReport, "ScanReport must be non-null");

Review Comment:
   updated



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922336284


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReporter.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.metrics;
+
+import java.util.Map;
+import org.apache.iceberg.metrics.ScanReport.ScanMetrics;
+
+/**
+ * This interface defines the basic API for a Table Scan Reporter that can be used to collect and report different
+ * metrics during a Table scan. The general usage of the Scan Reporter is as following:
+ * <ul>
+ *   <li>startScan() to indicate that this is a new Scan so that metrics (and other things) can be initialized</li>
+ *   <li>use scanMetrics() to add recorded metrics during a Scan</li>
+ *   <li>reportScan() at the end of the Scan</li>
+ * </ul>
+ */
+public interface ScanReporter {
+
+  /**
+   * Indicates that a new Scan is about to be performed, where implementors should initialize the {@link ScanMetrics},
+   * which can then be used to add recorded metrics to during a Scan.
+   */
+  void startScan();
+
+  /**
+   * Indicates that a Scan is done by reporting a {@link ScanReport}. A {@link ScanReport} is usually directly derived
+   * from a {@link ScanMetrics} instance.
+   *
+   * @param scanReport The {@link ScanReport} to report.
+   */
+  void reportScan(ScanReport scanReport);
+
+  /**
+   * The current {@link ScanMetrics} that can be used during a table scan to add metrics to.
+   *
+   * @return The current {@link ScanMetrics} that can be used during a table scan to add metrics to.
+   */
+  ScanMetrics scanMetrics();
+
+  /**
+   * Can be used to provide additional properties to the underlying {@link ScanReporter} implementation.
+   *
+   * @param properties The properties to pass to the underlying {@link ScanReporter} implementation.
+   */
+  void initialize(Map<String, String> properties);

Review Comment:
   I would probably remove customization for now. We can add this later.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921352784


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {

Review Comment:
   A lot of other frameworks implement `AutoCloseable` so that you can use try-with-resources to stop the timer:
   
   ```java
     List<DataFile> files;
     try (Timer t = context.timer("manfiest-read", TimeUnit.MICROSECONDS).start()) {
       this.files = ManifestFiles.read(...);
     }
   ```



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921892961


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);

Review Comment:
   I was broadly following how Micrometer defines their [Timer](https://www.javadoc.io/doc/io.micrometer/micrometer-core/1.0.3/io/micrometer/core/instrument/Timer.html) interface and I had the exact same thought as you mentioned here. 
   That being said, `time()` makes also sense to me as a verb for measuring a Runnable/Callable/Supplier, but I think `record(long amount, TimeUnit unit)` / `record(Duration duration)` reads nicer than `time(long amount, TimeUnit unit)` / `time(Duration duration)` and in order to keep method names aligned I decided to go with `record()` across the board. 
   



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922013829


##########
api/src/main/java/org/apache/iceberg/metrics/LongCounter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.metrics;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A default {@link org.apache.iceberg.metrics.MetricsContext.Counter} implementation that uses a {@link Long} to count
+ * events.
+ */
+class LongCounter implements MetricsContext.Counter<Long> {
+  private final AtomicLong counter;
+
+  LongCounter() {
+    counter = new AtomicLong(0L);
+  }
+
+  @Override
+  public void increment() {
+    counter.incrementAndGet();
+  }
+
+  @Override
+  public void increment(Long amount) {
+    counter.addAndGet(amount);

Review Comment:
   good catch, that makes sense. Updated the code to reflect that



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922365027


##########
core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java:
##########
@@ -34,7 +35,7 @@
 class IncrementalDataTableScan extends DataTableScan {
 
   IncrementalDataTableScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
-    super(ops, table, schema, context.useSnapshotId(null));
+    super(ops, table, schema, context.useSnapshotId(null), new LoggingScanReporter());

Review Comment:
   Rather than changing so many method signatures, why not add `ScanReporter` to the `TableScanContext`?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922363887


##########
core/src/main/java/org/apache/iceberg/BaseTableScan.java:
##########
@@ -116,9 +133,20 @@ public CloseableIterable<FileScanTask> planFiles() {
           ExpressionUtil.toSanitizedString(filter()));
 
       Listeners.notifyAll(new ScanEvent(table().name(), snapshot.snapshotId(), filter(), schema()));
-
-      return doPlanFiles();
-
+      scanReporter().startScan();
+      Timer.Sample sample = scanReporter().scanMetrics().totalScanDuration().start();

Review Comment:
   Tying the lifecycle of scan metrics to the scan reporter means that all of the scans created through this chain will have the same metrics collector, which is a correctness problem.
   
   In the Java library, the `TableScan` interface uses a refinement pattern. Each `TableScan` is independent and you get a new table scan that has additional options when you call the refinement methods, like `select` or `filter`. For example,
   
   ```java
   TableScan fullTableScan = table.newScan();
   TableScan yesterday = fullTableScan.filter(Expressions.equal("date_col", yesterday()));
   TableScan loginsYesterday = yesterday.filter(Expressions.equal("event_type", "login"));
   
   CloseableIterable<FileScanTask> yesterdayTasks = yesterday.planFiles();
   CloseableIterable<FileScanTask> loginTasks = loginsYesterday.planFiles();
   ```
   
   Two scans are planned, but the metrics for both would use the same `ScanMetrics`.
   
   Above, I suggested separating `ScanMetrics` from the reporter, which would solve this problem. Not all scans are planned, so you wouldn't want to eagerly create metrics, but the ones that are used should have a new one:
   
   ```java
     protected ScanMetrics metrics() {
       if (scanMetrics == null) {
         this.scanMetrics = scanReporter.newScanMetrics();
       }
   
       return scanMetrics;
     }
   
     public CloseableIterable<FileScanTask> planFiles(...) {
       ScanMetrics metrics = metrics();
       ...
     }
   ```



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922348325


##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -59,6 +76,10 @@ protected TableScanContext context() {
     return context;
   }
 
+  protected ScanReporter scanReporter() {

Review Comment:
   Does this need to be exposed publicly?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r921883294


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.StringJoiner;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> current = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public void startTimer() {
+    boolean updated = current.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+  }
+
+  @Override
+  public void stopTimer() {
+    Stopwatch stopwatch = current.getAndSet(null);
+    Preconditions.checkState(null != stopwatch, "startTimer() was not called");
+    record(stopwatch.stop().elapsed(defaultTimeUnit), defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      duration = duration.plus(amount, toChronoUnit(unit));
+      count.incrementAndGet();
+    }
+  }
+
+  @Override
+  public <T> T record(Supplier<T> supplier) {
+    startTimer();
+    try {
+      return supplier.get();
+    } finally {
+      stopTimer();
+    }
+  }
+
+  @Override
+  public <T> T recordCallable(Callable<T> callable) throws Exception {
+    startTimer();
+    try {
+      return callable.call();
+    } finally {
+      stopTimer();
+    }
+  }
+
+  @Override
+  public void record(Runnable runnable) {
+    startTimer();
+    try {
+      runnable.run();
+    } finally {
+      stopTimer();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", DefaultTimer.class.getSimpleName() + "[", "]")
+        .add("count=" + count)
+        .add("duration=" + duration)
+        .toString();
+  }

Review Comment:
   agreed, that looks much nicer



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922334117


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalScanDuration() {
+    return totalScanDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalScanDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFilesCount().count().orElse(-1),

Review Comment:
   Why would a count not default to 0?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922324472


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public CloseableSample startCloseable() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");

Review Comment:
   I think this needs to be updated. Now that it is using a `Sample` to hold the stopwatch, there's no need to fail if it is called concurrently.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922330139


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalScanDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalScanDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalScanDuration = totalScanDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {

Review Comment:
   When would a scan add data files?



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r922335234


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReporter.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.metrics;
+
+import java.util.Map;
+import org.apache.iceberg.metrics.ScanReport.ScanMetrics;
+
+/**
+ * This interface defines the basic API for a Table Scan Reporter that can be used to collect and report different
+ * metrics during a Table scan. The general usage of the Scan Reporter is as following:
+ * <ul>
+ *   <li>startScan() to indicate that this is a new Scan so that metrics (and other things) can be initialized</li>
+ *   <li>use scanMetrics() to add recorded metrics during a Scan</li>
+ *   <li>reportScan() at the end of the Scan</li>
+ * </ul>
+ */
+public interface ScanReporter {
+
+  /**
+   * Indicates that a new Scan is about to be performed, where implementors should initialize the {@link ScanMetrics},
+   * which can then be used to add recorded metrics to during a Scan.
+   */
+  void startScan();

Review Comment:
   Similar to the comment above, this would be `startScanPlanning`.
   
   I'm also not sure that the `ScanReporter` itself should be a timer. That mixes more responsibility than necessary. I would probably just add a timer for total planning time to the scan metrics.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r933638567


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/** A Table Scan report that contains all relevant information from a Table Scan. */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final ScanMetrics scanMetrics;
+
+  private ScanReport(
+      String tableName,
+      long snapshotId,
+      Expression filter,
+      Schema projection,
+      ScanMetrics scanMetrics) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.scanMetrics = scanMetrics;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public ScanMetrics scanMetrics() {
+    return scanMetrics;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("scanMetrics", scanMetrics)
+        .toString();
+  }
+
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {}
+
+    public Builder withTableName(String newTableName) {
+      this.tableName = newTableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long newSnapshotId) {
+      this.snapshotId = newSnapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression newFilter) {
+      this.filter = newFilter;
+      return this;
+    }
+
+    public Builder withProjection(Schema newProjection) {
+      this.projection = newProjection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics newScanMetrics) {
+      this.scanMetrics = newScanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "Invalid table name: null");
+      Preconditions.checkArgument(null != filter, "Invalid expression filter: null");
+      Preconditions.checkArgument(null != projection, "Invalid schema projection: null");
+      Preconditions.checkArgument(null != scanMetrics, "Invalid scan metrics: null");
+      return new ScanReport(tableName, snapshotId, filter, projection, scanMetrics);
+    }
+  }
+
+  /** Carries all metrics for a particular scan */
+  public static class ScanMetrics {
+    public static final ScanMetrics NOOP = new ScanMetrics(MetricsContext.nullMetrics());
+    private final Counter<Integer> resultDataFiles;
+    private final Counter<Integer> totalDataManifests;
+    private final Counter<Integer> totalDeleteManifests;
+    private final Counter<Integer> scannedDataManifests;
+    private final Counter<Integer> scannedDeleteManifests;
+    private final Counter<Integer> totalDataFiles;
+    private final Counter<Integer> totalDeleteFiles;
+    private final Timer totalPlanningDuration;
+    private final Counter<Long> totalFileSizeInBytes;
+    private final Counter<Integer> indexedDeleteFiles;
+    private final Counter<Integer> resultDeleteFiles;
+    private final Counter<Integer> skippedDataManifests;
+    private final Counter<Integer> skippedDeleteManifests;
+    private final Counter<Integer> skippedDataFiles;
+    private final Counter<Integer> skippedDeleteFiles;
+
+    public ScanMetrics(MetricsContext metricsContext) {
+      Preconditions.checkArgument(null != metricsContext, "Invalid metrics context: null");
+      this.totalDataManifests =
+          metricsContext.counter("totalDataManifests", Integer.class, MetricsContext.Unit.COUNT);
+      this.totalDeleteManifests =
+          metricsContext.counter("totalDeleteManifests", Integer.class, MetricsContext.Unit.COUNT);
+      this.resultDataFiles =
+          metricsContext.counter("resultDataFiles", Integer.class, MetricsContext.Unit.COUNT);
+      this.scannedDataManifests =
+          metricsContext.counter("scannedDataManifests", Integer.class, MetricsContext.Unit.COUNT);
+      this.scannedDeleteManifests =
+          metricsContext.counter(
+              "scannedDeleteManifests", Integer.class, MetricsContext.Unit.COUNT);
+      this.totalDataFiles =
+          metricsContext.counter("totalDataFiles", Integer.class, MetricsContext.Unit.COUNT);
+      this.totalDeleteFiles =
+          metricsContext.counter("totalDeleteFiles", Integer.class, MetricsContext.Unit.COUNT);

Review Comment:
   I think this needs to be implemented. Currently, the code counts deleted files rather than live files in delete manifests. You should be able to update this in `DeleteFileIndex` by passing `scanMetrics` to the `ManifestReader`. I think that will also fix `skippedDeleteFiles`.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r933636831


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -184,12 +192,14 @@ public <T extends ScanTask> CloseableIterable<T> plan(CreateTasksFunction<T> cre
                 specId -> {
                   PartitionSpec spec = specsById.get(specId);
                   ResidualEvaluator residuals = residualCache.get(specId);
-                  return new TaskContext(spec, deleteFiles, residuals, dropStats);
+                  return new TaskContext(spec, deleteFiles, residuals, dropStats, scanMetrics);
                 });
 
     Iterable<CloseableIterable<T>> tasks =
         entries(
             (manifest, entries) -> {
+              scanMetrics.totalDataFiles().increment(manifest.addedFilesCount());
+              scanMetrics.totalDeleteFiles().increment(manifest.deletedFilesCount());

Review Comment:
   Delete files and deleted files are distinct. This is the number of deleted entries in the manifest, not the number of delete files.
   
   In addition, existing data files will be used in the scan, not just added data files. Since this is counting inside of `ManifestGroup`, we need to make sure that all of the returned files are accounted for. There are two options:
   1. Put these updates in `entries` where `ignoreDeleted` and `ignoreExisting` are checked
   2. Add the total data files counter to the manifest reader, like you currently have for skipped files
   
   I prefer option 2. I would opt to have simpler hooks instead of trying to count in bulk like this using metadata counts at the right place. That's probably going to be the most reliable way to get this data.



-- 
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-unsubscribe@iceberg.apache.org

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] rdblue commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r931586411


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final ScanMetrics scanMetrics;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      ScanMetrics scanMetrics) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.scanMetrics = scanMetrics;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public ScanMetrics scanMetrics() {
+    return scanMetrics;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("scanMetrics", scanMetrics)
+        .toString();
+  }
+
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String newTableName) {
+      this.tableName = newTableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long newSnapshotId) {
+      this.snapshotId = newSnapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression newFilter) {
+      this.filter = newFilter;
+      return this;
+    }
+
+    public Builder withProjection(Schema newProjection) {
+      this.projection = newProjection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics newScanMetrics) {

Review Comment:
   Shouldn't this be created by the builder, rather than passed in?



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923015754


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer
+   */
+  void startTimer();
+
+  /**
+   * Stops the timer and records the total duration up until {@link Timer#startTimer()} was called.
+   */
+  void stopTimer();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);

Review Comment:
   ok renamed those 3 methods to `time(...)`



##########
api/src/main/java/org/apache/iceberg/metrics/DefaultMetricsContext.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A default {@link MetricsContext} implementation that uses native Java counters/timers.
+ */
+public class DefaultMetricsContext implements MetricsContext {
+
+  @Override
+  public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
+    if (Integer.class.equals(type)) {
+      return (Counter<T>) new IntCounter();
+    }
+    if (Long.class.equals(type)) {

Review Comment:
   fixed



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923170341


##########
core/src/main/java/org/apache/iceberg/CatalogUtil.java:
##########
@@ -346,4 +347,42 @@ public static void configureHadoopConf(Object maybeConfigurable, Object conf) {
 
     setConf.invoke(conf);
   }
+
+  /**
+   * Load a custom {@link ScanReporter} implementation.
+   * <p>
+   * The implementation must have a no-arg constructor. {@link ScanReporter#initialize(Map properties)} is called to
+   * complete the initialization.
+   *
+   * @param impl       full class name of a custom {@link ScanReporter} implementation
+   * @param properties used to initialize the {@link ScanReporter} implementation
+   * @return An initialized {@link ScanReporter}.
+   * @throws IllegalArgumentException if class path not found or right constructor not found or the loaded class cannot
+   *                                  be cast to the given interface type
+   */
+  public static ScanReporter loadScanReporter(String impl, Map<String, String> properties) {

Review Comment:
   ok removed it



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r936892488


##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);
+    assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
+
+    // should be two files

Review Comment:
   yeah the initial version of the test was mainly for me to better understand how scan planning works. Reduced it to 3 files and 1 file



##########
data/src/test/java/org/apache/iceberg/TestScanPlanningAndReporting.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ManifestEntry.Status;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.metrics.LoggingScanReporter;
+import org.apache.iceberg.metrics.ScanReport;
+import org.apache.iceberg.metrics.ScanReporter;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestScanPlanningAndReporting extends TableTestBase {
+
+  private final TestScanReporter reporter = new TestScanReporter();
+
+  public TestScanPlanningAndReporting() {
+    super(2);
+  }
+
+  @Test
+  public void testScanPlanningWithReport() throws IOException {
+    String tableName = "simple-scan-planning";
+    Table tbl = createTableWithCustomRecords(tableName);
+    TableScan tableScan = tbl.newScan();
+
+    // should be 3 files
+    try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+      fileScanTasks.forEach(task -> {});
+    }
+
+    ScanReport scanReport = reporter.lastReport();
+    assertThat(scanReport).isNotNull();
+
+    assertThat(scanReport.tableName()).isEqualTo(tableName);
+    assertThat(scanReport.snapshotId()).isEqualTo(1L);
+    assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue());
+    assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration())
+        .isGreaterThan(Duration.ZERO);
+    assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(3);
+    assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().skippedDataManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(1);
+    assertThat(scanReport.scanMetrics().totalDeleteManifests().value()).isEqualTo(0);
+    assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isGreaterThan(0L);

Review Comment:
   done



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r934202889


##########
core/src/main/java/org/apache/iceberg/ManifestGroup.java:
##########
@@ -325,6 +351,9 @@ private static CloseableIterable<FileScanTask> createFileScanTasks(
         entry -> {
           DataFile dataFile = entry.file().copy(ctx.shouldKeepStats());
           DeleteFile[] deleteFiles = ctx.deletes().forEntry(entry);
+          ctx.scanMetrics().totalFileSizeInBytes().increment(dataFile.fileSizeInBytes());

Review Comment:
   good point, added



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929588827


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")
+  public static class Builder {
+    private String tableName;
+    private long snapshotId = -1L;
+    private Expression filter;
+    private Schema projection;
+    private ScanMetrics scanMetrics;
+
+    private Builder() {
+    }
+
+    public Builder withTableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder withSnapshotId(long snapshotId) {
+      this.snapshotId = snapshotId;
+      return this;
+    }
+
+    public Builder withFilter(Expression filter) {
+      this.filter = filter;
+      return this;
+    }
+
+    public Builder withProjection(Schema projection) {
+      this.projection = projection;
+      return this;
+    }
+
+    public Builder fromScanMetrics(ScanMetrics scanMetrics) {
+      this.scanMetrics = scanMetrics;
+      return this;
+    }
+
+    public ScanReport build() {
+      Preconditions.checkArgument(null != tableName, "TableName must be non-null");
+      Preconditions.checkArgument(null != filter, "Expression filter must be non-null");
+      Preconditions.checkArgument(null != projection, "Schema projection must be non-null");
+      Preconditions.checkArgument(null != scanMetrics, "ScanMetrics must be non-null");
+      return new ScanReport(tableName, snapshotId, filter, projection,
+          scanMetrics.matchingDataFiles().count().orElse(-1),

Review Comment:
   I'm rather thinking of the case where someone would want to send the `ScanReport` over the wire. For example, with the REST catalog we might want to send the `ScanReport` to an endpoint via a `POST`. Having actual `Counters` / `Timers` in the `ScanReport` makes things difficult. Does that make sense? 



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r929591721


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.metrics;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.metrics.MetricsContext.Counter;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * A Table Scan report that contains all relevant information from a Table Scan.
+ */
+public class ScanReport implements Serializable {
+
+  private final String tableName;
+  private final long snapshotId;
+  private final Expression filter;
+  private final Schema projection;
+  private final int matchingDataFiles;
+  private final int matchingDataManifests;
+  private final int totalDataManifestsRead;
+  private final int addedDataFiles;
+  private final int deletedDataFiles;
+  private final Duration totalPlanningDuration;
+  private final long totalFileSizeBytes;
+
+  private ScanReport(
+      String tableName, long snapshotId, Expression filter, Schema projection,
+      int matchingDataFiles, int matchingDataManifests, int totalDataManifestsRead,
+      int addedDataFiles, int deletedDataFiles, Duration totalPlanningDuration,
+      long totalFileSizeBytes) {
+    this.tableName = tableName;
+    this.snapshotId = snapshotId;
+    this.filter = filter;
+    this.projection = projection;
+    this.matchingDataFiles = matchingDataFiles;
+    this.matchingDataManifests = matchingDataManifests;
+    this.totalDataManifestsRead = totalDataManifestsRead;
+    this.addedDataFiles = addedDataFiles;
+    this.deletedDataFiles = deletedDataFiles;
+    this.totalPlanningDuration = totalPlanningDuration;
+    this.totalFileSizeBytes = totalFileSizeBytes;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public Schema projection() {
+    return projection;
+  }
+
+  public int matchingDataFiles() {
+    return matchingDataFiles;
+  }
+
+  public int matchingDataManifests() {
+    return matchingDataManifests;
+  }
+
+  public int totalDataManifestsRead() {
+    return totalDataManifestsRead;
+  }
+
+  public int addedDataFiles() {
+    return addedDataFiles;
+  }
+
+  public int deletedDataFiles() {
+    return deletedDataFiles;
+  }
+
+  public Duration totalPlanningDuration() {
+    return totalPlanningDuration;
+  }
+
+  public long totalFileSizeBytes() {
+    return totalFileSizeBytes;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("tableName", tableName)
+        .add("snapshotId", snapshotId)
+        .add("filter", filter)
+        .add("projection", projection)
+        .add("matchingDataFiles", matchingDataFiles)
+        .add("matchingDataManifests", matchingDataManifests)
+        .add("totalDataManifestsRead", totalDataManifestsRead)
+        .add("addedDataFiles", addedDataFiles)
+        .add("deletedDataFiles", deletedDataFiles)
+        .add("totalScanDuration", totalPlanningDuration)
+        .add("totalFileSizeBytes", totalFileSizeBytes)
+        .toString();
+  }
+
+  @SuppressWarnings("HiddenField")

Review Comment:
   ok updated, but to be honest I don't feel like having the `HiddenField` checkstyle rule is actually that useful :)



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923008179


##########
api/src/main/java/org/apache/iceberg/metrics/Timer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.metrics;
+
+import java.time.Duration;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * Generalized Timer interface for creating telemetry related instances for measuring duration of operations.
+ */
+public interface Timer {
+
+  /**
+   * The number of times {@link Timer#record(Duration)} was called.
+   *
+   * @return The number of times {@link Timer#record(Duration)} was called.
+   */
+  long count();
+
+  /**
+   * The total duration that was recorded.
+   *
+   * @return The total duration that was recorded.
+   */
+  Duration totalDuration();
+
+  /**
+   * Starts the timer and returns a {@link Sample}. Call {@link Sample#stop(Timer)} to complete the timing.
+   *
+   * @return A timing {@link Sample} with the start time recorded.
+   */
+  Sample start();
+
+  CloseableSample startCloseable();
+
+  /**
+   * Records a custom amount in the given time unit.
+   *
+   * @param amount The amount to record
+   * @param unit   The time unit of the amount
+   */
+  void record(long amount, TimeUnit unit);
+
+  /**
+   * The duration to record
+   *
+   * @param duration The duration to record
+   */
+  default void record(Duration duration) {
+    record(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Executes and measures the given {@link Runnable} instance.
+   *
+   * @param runnable The {@link Runnable} to execute and measure.
+   */
+  void record(Runnable runnable);
+
+  /**
+   * Executes and measures the given {@link Callable} and returns its result.
+   *
+   * @param callable The {@link Callable} to execute and measure.
+   * @param <T>      The type of the {@link Callable}
+   * @return The result of the underlying {@link Callable}.
+   * @throws Exception In case the {@link Callable} fails.
+   */
+  <T> T recordCallable(Callable<T> callable) throws Exception;
+
+  /**
+   * Gets the result from the given {@link Supplier} and measures its execution time.
+   *
+   * @param supplier The {@link Supplier} to execute and measure.
+   * @param <T>      The type of the {@link Supplier}.
+   * @return The result of the underlying {@link Supplier}.
+   */
+  <T> T record(Supplier<T> supplier);
+
+  /**
+   * A timing sample that carries internal state about the Timer's start position. The timing can be completed by
+   * calling {@link Sample#stop(Timer)}.
+   */
+  interface Sample {

Review Comment:
   `Timed` sounds good to me. Updated



-- 
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-unsubscribe@iceberg.apache.org

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] nastra commented on a diff in pull request #5268: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#discussion_r923156357


##########
api/src/test/java/org/apache/iceberg/io/TestCloseableIterable.java:
##########
@@ -82,4 +85,61 @@ public void testConcateWithEmptyIterables() {
         NoSuchElementException.class,
         () -> Iterables.getLast(concat5));
   }
+
+  @Test
+  public void testWithCompletionRunnable() throws IOException {
+    AtomicInteger completionCounter = new AtomicInteger(0);
+    List<Integer> items = Lists.newArrayList(1, 2, 3, 4, 5);
+    Assertions.assertThatThrownBy(() ->
+                    CloseableIterable.whenComplete(CloseableIterable.combine(items, () -> { }), null))
+            .isInstanceOf(NullPointerException.class)
+            .hasMessage("Cannot execute a null Runnable after completion");

Review Comment:
   fixed



##########
api/src/main/java/org/apache/iceberg/metrics/ScanReporter.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.metrics;
+
+import java.util.Map;
+import org.apache.iceberg.metrics.ScanReport.ScanMetrics;
+
+/**
+ * This interface defines the basic API for a Table Scan Reporter that can be used to collect and report different
+ * metrics during a Table scan. The general usage of the Scan Reporter is as following:
+ * <ul>
+ *   <li>startScan() to indicate that this is a new Scan so that metrics (and other things) can be initialized</li>
+ *   <li>use scanMetrics() to add recorded metrics during a Scan</li>
+ *   <li>reportScan() at the end of the Scan</li>
+ * </ul>
+ */
+public interface ScanReporter {
+
+  /**
+   * Indicates that a new Scan is about to be performed, where implementors should initialize the {@link ScanMetrics},
+   * which can then be used to add recorded metrics to during a Scan.
+   */
+  void startScan();
+
+  /**
+   * Indicates that a Scan is done by reporting a {@link ScanReport}. A {@link ScanReport} is usually directly derived
+   * from a {@link ScanMetrics} instance.
+   *
+   * @param scanReport The {@link ScanReport} to report.
+   */
+  void reportScan(ScanReport scanReport);
+
+  /**
+   * The current {@link ScanMetrics} that can be used during a table scan to add metrics to.
+   *
+   * @return The current {@link ScanMetrics} that can be used during a table scan to add metrics to.
+   */
+  ScanMetrics scanMetrics();
+
+  /**
+   * Can be used to provide additional properties to the underlying {@link ScanReporter} implementation.
+   *
+   * @param properties The properties to pass to the underlying {@link ScanReporter} implementation.
+   */
+  void initialize(Map<String, String> properties);

Review Comment:
   ok removed it



-- 
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-unsubscribe@iceberg.apache.org

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] findepi commented on pull request #5268: API/Core: Initial Table Scan Reporting support

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #5268:
URL: https://github.com/apache/iceberg/pull/5268#issuecomment-1361614786

   Just FYI, this seems to have inflated logs quite noticeably (https://github.com/trinodb/trino/issues/15492).
   I wonder whether INFO-level logging should be the default behavior.
   Wouldn't it be better to have these logged at DEBUG by default?
   


-- 
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-unsubscribe@iceberg.apache.org

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