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/09/16 21:01:31 UTC

[GitHub] [iceberg] nastra opened a new pull request, #5780: API/Core: Make ScanReport and its related classes Immutable

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

   * Use true immutable objects that are type-safe, thread-safe, null-safe
   * Get builder classes for free
   
   This is relying on https://immutables.github.io/ (Apache License 2.0), which allows generating immutable objects and builders via annotation processing.
   * Immutable objects are serialization ready (including JSON and its binary forms)
   * Supports lazy, derived and optional attributes
   * Immutable objects are constructed once, in a consistent state, and can be safely shared
     * Will fail if mandatory attributes are missing
     * Cannot be sneakily modified when passed to other code
   * Immutable objects are naturally thread-safe and can therefore be safely shared among threads
     * No excessive copying
     * No excessive synchronization
   * Object definitions are pleasant to write and read
     * No boilerplate setter and getters
     * No ugly IDE-generated hashCode, equals and toString methods that end up being stored in source control.
   
   Note that we are specifically preventing people from using Jackson-related annotations (`@JsonSerialize` & `@JsonDeserialize`) in order to avoid potential runtime library dependency issues where a clashing jackson lib would prevent jackson-related annotations to be used/processed.


-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -419,86 +154,60 @@ public static class ScanMetrics {
     public static final String TOTAL_FILE_SIZE_IN_BYTES = "total-file-size-in-bytes";
     public static final String TOTAL_DELETE_FILE_SIZE_IN_BYTES = "total-delete-file-size-in-bytes";
     public static final String SKIPPED_DATA_MANIFESTS = "skipped-data-manifests";
-    private final Timer totalPlanningDuration;
-    private final Counter resultDataFiles;
-    private final Counter resultDeleteFiles;
-    private final Counter totalDataManifests;
-    private final Counter totalDeleteManifests;
-    private final Counter scannedDataManifests;
-    private final Counter skippedDataManifests;
-    private final Counter totalFileSizeInBytes;
-    private final Counter totalDeleteFileSizeInBytes;
-
-    public ScanMetrics(MetricsContext metricsContext) {
-      Preconditions.checkArgument(null != metricsContext, "Invalid metrics context: null");
-      this.totalPlanningDuration =
-          metricsContext.timer(TOTAL_PLANNING_DURATION, TimeUnit.NANOSECONDS);
-      this.resultDataFiles = metricsContext.counter(RESULT_DATA_FILES, MetricsContext.Unit.COUNT);
-      this.resultDeleteFiles =
-          metricsContext.counter(RESULT_DELETE_FILES, MetricsContext.Unit.COUNT);
-      this.scannedDataManifests =
-          metricsContext.counter(SCANNED_DATA_MANIFESTS, MetricsContext.Unit.COUNT);
-      this.totalDataManifests =
-          metricsContext.counter(TOTAL_DATA_MANIFESTS, MetricsContext.Unit.COUNT);
-      this.totalDeleteManifests =
-          metricsContext.counter(TOTAL_DELETE_MANIFESTS, MetricsContext.Unit.COUNT);
-      this.totalFileSizeInBytes =
-          metricsContext.counter(TOTAL_FILE_SIZE_IN_BYTES, MetricsContext.Unit.BYTES);
-      this.totalDeleteFileSizeInBytes =
-          metricsContext.counter(TOTAL_DELETE_FILE_SIZE_IN_BYTES, MetricsContext.Unit.BYTES);
-      this.skippedDataManifests =
-          metricsContext.counter(SKIPPED_DATA_MANIFESTS, MetricsContext.Unit.COUNT);
+
+    public static ScanMetrics noop() {
+      return ScanMetrics.of(MetricsContext.nullMetrics());
     }
 
+    public abstract MetricsContext metricsContext();
+
+    @Value.Derived
     public Timer totalPlanningDuration() {
-      return totalPlanningDuration;
+      return metricsContext().timer(TOTAL_PLANNING_DURATION, TimeUnit.NANOSECONDS);

Review Comment:
   I see the `MetricsContext` parameter here less as a "this class needs to keep track of this" but rather "this is what's required to create an instance of this class". Your comment is perfectly valid here and I initially had the exact same thought. 
   Therefore it make sense to have a `ScanMetrics.from(MetricsContext)`. In fact we have a `ScanMetrics.of(MetricsContext)` that does exactly that. 
   
   We could go ahead and change `ScanMetrics.of(...)` to:
   ```
   public static ScanMetrics of(MetricsContext metricsContext) {
         return ImmutableScanMetrics.builder()
             .metricsContext(metricsContext)
             .totalPlanningDuration(
                 metricsContext.timer(TOTAL_PLANNING_DURATION, TimeUnit.NANOSECONDS))
             .resultDataFiles(metricsContext.counter(RESULT_DATA_FILES, MetricsContext.Unit.COUNT))
             .resultDeleteFiles(metricsContext.counter(RESULT_DELETE_FILES, MetricsContext.Unit.COUNT))
             .scannedDataManifests(
                 metricsContext.counter(SCANNED_DATA_MANIFESTS, MetricsContext.Unit.COUNT))
             .totalDataManifests(
                 metricsContext.counter(TOTAL_DATA_MANIFESTS, MetricsContext.Unit.COUNT))
             .totalDeleteManifests(
                 metricsContext.counter(TOTAL_DELETE_MANIFESTS, MetricsContext.Unit.COUNT))
             .totalFileSizeInBytes(
                 metricsContext.counter(TOTAL_FILE_SIZE_IN_BYTES, MetricsContext.Unit.BYTES))
             .totalDeleteFileSizeInBytes(
                 metricsContext.counter(TOTAL_DELETE_FILE_SIZE_IN_BYTES, MetricsContext.Unit.BYTES))
             .skippedDataManifests(
                 metricsContext.counter(SKIPPED_DATA_MANIFESTS, MetricsContext.Unit.COUNT))
             .build();
       }
   ```
   It would produce the exact same results. The only reason I went with `@Value.Derived` is because it does not allow manual creation of metric instances.
   `ScanMetrics.of(...)` is just a utility method and it doesn't prevent anyone from using the Builder directly. Using `@Value.Derived` in this context means essentially that all those metric instances are derived from `MetricsContext` but **cannot be manually set**, thus providing a builder where only `ImmutableScanMetrics.builder()        .metricsContext(metricsContext).build()` can be set.
   The lazy initialization of metrics that `@Value.Derived` provides is just a nice thing to have that we get here, but my main motivation was to prevent the different counters/timers to be manually set. This gives us the exact same semantics we had in the previous `ScanMetrics` API, since all the metrics were initialized in the constructor.
   
   



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
api/src/test/java/org/apache/iceberg/metrics/TestScanReport.java:
##########
@@ -66,11 +83,12 @@ public void fromEmptyScanMetrics() {
     Schema projection =
         new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1"));
     ScanReport scanReport =
-        ScanReport.builder()
-            .withTableName(tableName)
-            .withProjection(projection)
-            .withFilter(Expressions.alwaysTrue())
-            .fromScanMetrics(ScanReport.ScanMetrics.NOOP)
+        ImmutableScanReport.builder()
+            .tableName(tableName)
+            .snapshotId(-1L)

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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
api/src/test/java/org/apache/iceberg/metrics/TestScanReport.java:
##########
@@ -66,11 +83,12 @@ public void fromEmptyScanMetrics() {
     Schema projection =
         new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1"));
     ScanReport scanReport =
-        ScanReport.builder()
-            .withTableName(tableName)
-            .withProjection(projection)
-            .withFilter(Expressions.alwaysTrue())
-            .fromScanMetrics(ScanReport.ScanMetrics.NOOP)
+        ImmutableScanReport.builder()
+            .tableName(tableName)
+            .snapshotId(-1L)

Review Comment:
   we could also add validation to make sure snapshotId is always > 0. All that would be required is adding the below method to `ScanReport`:
   
   ```
     @Check
     default void check() {
       Preconditions.checkArgument(snapshotId() > 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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
versions.props:
##########
@@ -27,6 +27,7 @@ com.google.cloud:libraries-bom = 24.1.0
 org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0
 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0
 com.emc.ecs:object-client-bundle = 3.3.2
+org.immutables:value = 2.9.0

Review Comment:
   It looks like this pulls in another copy of Guava? https://search.maven.org/artifact/org.immutables/value/2.9.2/jar



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java:
##########
@@ -45,86 +45,54 @@ public void nullMetrics() {
   @Test
   public void missingFields() {
     Assertions.assertThat(ScanMetricsResultParser.fromJson("{}"))
-        .isEqualTo(new ScanMetricsResult(null, null, null, null, null, null, null, null, null));
-
-    TimerResult totalPlanningDuration = new TimerResult(TimeUnit.HOURS, Duration.ofHours(10), 3L);
-    CounterResult resultDataFiles = new CounterResult(Unit.COUNT, 5L);
-    CounterResult resultDeleteFiles = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalDeleteManifests = new CounterResult(Unit.COUNT, 0L);
-    CounterResult scannedDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult skippedDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalFileSizeInBytes = new CounterResult(Unit.BYTES, 1069L);
+        .isEqualTo(ImmutableScanMetricsResult.builder().build());
 
+    ImmutableScanMetricsResult scanMetricsResult =

Review Comment:
   it's actually not a Builder instance here but the actual ScanMetricsResult instance and we're making a copy of it and then just update a single metric:
   
   ```
   ImmutableScanMetricsResult scanMetricsResult = 
           ImmutableScanMetricsResult.builder()
           .totalPlanningDuration(TimerResult.of(TimeUnit.HOURS, Duration.ofHours(10), 3L))
           .build();
   
   // copy whatever was already set and then only configure result-data-files
   scanMetricsResult = scanMetricsResult.withResultDataFiles(CounterResult.of(Unit.COUNT, 5L));
   ```
   
   We could also have used `scanMetricsResult = ImmutableScanMetricsResult.copyOf(scanMetricsResult).withResultDataFiles(CounterResult.of(Unit.COUNT, 5L))` which is slightly longer but a bit more explicit and indicates that we're making a copy 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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java:
##########
@@ -45,86 +45,54 @@ public void nullMetrics() {
   @Test
   public void missingFields() {
     Assertions.assertThat(ScanMetricsResultParser.fromJson("{}"))
-        .isEqualTo(new ScanMetricsResult(null, null, null, null, null, null, null, null, null));
-
-    TimerResult totalPlanningDuration = new TimerResult(TimeUnit.HOURS, Duration.ofHours(10), 3L);
-    CounterResult resultDataFiles = new CounterResult(Unit.COUNT, 5L);
-    CounterResult resultDeleteFiles = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalDeleteManifests = new CounterResult(Unit.COUNT, 0L);
-    CounterResult scannedDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult skippedDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalFileSizeInBytes = new CounterResult(Unit.BYTES, 1069L);
+        .isEqualTo(ImmutableScanMetricsResult.builder().build());
 
+    ImmutableScanMetricsResult scanMetricsResult =

Review Comment:
   `resultBuilder`?



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
build.gradle:
##########
@@ -230,6 +230,8 @@ project(':iceberg-api') {
   dependencies {
     implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
     compileOnly "com.google.errorprone:error_prone_annotations"
+    annotationProcessor "org.immutables:value"
+    implementation "org.immutables:value"

Review Comment:
   I thought you had said that this didn't require any additional runtime dependencies? We need to go through and validate the license and license documentation (NOTICE).



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
baseline.gradle:
##########
@@ -57,8 +57,8 @@ subprojects {
   pluginManager.withPlugin('com.palantir.baseline-error-prone') {
     tasks.withType(JavaCompile).configureEach {
       options.errorprone.errorproneArgs.addAll (
-          // error-prone is slow, don't run on tests and generated src
-          '-XepExcludedPaths:.*/(test|generated-src)/.*',
+          // error-prone is slow, don't run on tests/generated-src/generated
+          '-XepExcludedPaths:.*/(test|generated-src|generated)/.*',

Review Comment:
   yes this should be perfectly ok, it just needs a single refresh in the IDE for the generated classes to show 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] rdblue commented on a diff in pull request #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
api/src/test/java/org/apache/iceberg/metrics/TestScanReport.java:
##########
@@ -66,11 +83,12 @@ public void fromEmptyScanMetrics() {
     Schema projection =
         new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1"));
     ScanReport scanReport =
-        ScanReport.builder()
-            .withTableName(tableName)
-            .withProjection(projection)
-            .withFilter(Expressions.alwaysTrue())
-            .fromScanMetrics(ScanReport.ScanMetrics.NOOP)
+        ImmutableScanReport.builder()
+            .tableName(tableName)
+            .snapshotId(-1L)

Review Comment:
   Since this is not required (and probably should have been before) can we just set it to a reasonable value rather than passing through -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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
api/src/test/java/org/apache/iceberg/metrics/TestScanReport.java:
##########
@@ -66,11 +83,12 @@ public void fromEmptyScanMetrics() {
     Schema projection =
         new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1"));
     ScanReport scanReport =
-        ScanReport.builder()
-            .withTableName(tableName)
-            .withProjection(projection)
-            .withFilter(Expressions.alwaysTrue())
-            .fromScanMetrics(ScanReport.ScanMetrics.NOOP)
+        ImmutableScanReport.builder()
+            .tableName(tableName)
+            .snapshotId(-1L)

Review Comment:
   This should not allow -1 as a snapshot ID, right? Was it null before?



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
api/src/main/java/org/apache/iceberg/metrics/ScanReport.java:
##########
@@ -419,86 +154,60 @@ public static class ScanMetrics {
     public static final String TOTAL_FILE_SIZE_IN_BYTES = "total-file-size-in-bytes";
     public static final String TOTAL_DELETE_FILE_SIZE_IN_BYTES = "total-delete-file-size-in-bytes";
     public static final String SKIPPED_DATA_MANIFESTS = "skipped-data-manifests";
-    private final Timer totalPlanningDuration;
-    private final Counter resultDataFiles;
-    private final Counter resultDeleteFiles;
-    private final Counter totalDataManifests;
-    private final Counter totalDeleteManifests;
-    private final Counter scannedDataManifests;
-    private final Counter skippedDataManifests;
-    private final Counter totalFileSizeInBytes;
-    private final Counter totalDeleteFileSizeInBytes;
-
-    public ScanMetrics(MetricsContext metricsContext) {
-      Preconditions.checkArgument(null != metricsContext, "Invalid metrics context: null");
-      this.totalPlanningDuration =
-          metricsContext.timer(TOTAL_PLANNING_DURATION, TimeUnit.NANOSECONDS);
-      this.resultDataFiles = metricsContext.counter(RESULT_DATA_FILES, MetricsContext.Unit.COUNT);
-      this.resultDeleteFiles =
-          metricsContext.counter(RESULT_DELETE_FILES, MetricsContext.Unit.COUNT);
-      this.scannedDataManifests =
-          metricsContext.counter(SCANNED_DATA_MANIFESTS, MetricsContext.Unit.COUNT);
-      this.totalDataManifests =
-          metricsContext.counter(TOTAL_DATA_MANIFESTS, MetricsContext.Unit.COUNT);
-      this.totalDeleteManifests =
-          metricsContext.counter(TOTAL_DELETE_MANIFESTS, MetricsContext.Unit.COUNT);
-      this.totalFileSizeInBytes =
-          metricsContext.counter(TOTAL_FILE_SIZE_IN_BYTES, MetricsContext.Unit.BYTES);
-      this.totalDeleteFileSizeInBytes =
-          metricsContext.counter(TOTAL_DELETE_FILE_SIZE_IN_BYTES, MetricsContext.Unit.BYTES);
-      this.skippedDataManifests =
-          metricsContext.counter(SKIPPED_DATA_MANIFESTS, MetricsContext.Unit.COUNT);
+
+    public static ScanMetrics noop() {
+      return ScanMetrics.of(MetricsContext.nullMetrics());
     }
 
+    public abstract MetricsContext metricsContext();
+
+    @Value.Derived
     public Timer totalPlanningDuration() {
-      return totalPlanningDuration;
+      return metricsContext().timer(TOTAL_PLANNING_DURATION, TimeUnit.NANOSECONDS);

Review Comment:
   Wouldn't it be simpler just to use a `ScanMetrics.from(MetricsContext)` method so that `ScanMetrics` doesn't need to keep track of the context? The `from` method could create all of the metrics and pass them into the builder and we wouldn't need all the generated code for initialization and making sure `MetricsContext` is only called once per counter/timer. If we also make these required, then there should be no problem with them being null.



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
baseline.gradle:
##########
@@ -57,8 +57,8 @@ subprojects {
   pluginManager.withPlugin('com.palantir.baseline-error-prone') {
     tasks.withType(JavaCompile).configureEach {
       options.errorprone.errorproneArgs.addAll (
-          // error-prone is slow, don't run on tests and generated src
-          '-XepExcludedPaths:.*/(test|generated-src)/.*',
+          // error-prone is slow, don't run on tests/generated-src/generated
+          '-XepExcludedPaths:.*/(test|generated-src|generated)/.*',

Review Comment:
   Actually, after running build I do see the classes show up without reconfiguring. Looks like this should be 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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
versions.props:
##########
@@ -27,6 +27,7 @@ com.google.cloud:libraries-bom = 24.1.0
 org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0
 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0
 com.emc.ecs:object-client-bundle = 3.3.2
+org.immutables:value = 2.9.0

Review Comment:
   all dependencies that `immutables.value` requires are properly repackaged and the guava classes live under `org.immutables.value.internal.$guava$`, so this shouldn't cause any issues. However, I didn't realize that they just released a new version, so I'm upgrading this to `2.9.2`



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
baseline.gradle:
##########
@@ -57,8 +57,8 @@ subprojects {
   pluginManager.withPlugin('com.palantir.baseline-error-prone') {
     tasks.withType(JavaCompile).configureEach {
       options.errorprone.errorproneArgs.addAll (
-          // error-prone is slow, don't run on tests and generated src
-          '-XepExcludedPaths:.*/(test|generated-src)/.*',
+          // error-prone is slow, don't run on tests/generated-src/generated
+          '-XepExcludedPaths:.*/(test|generated-src|generated)/.*',

Review Comment:
   yes this should be perfectly ok and I've never had any issues with this in the past. It just needs a single refresh in the IDE for the generated classes to show 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] rdblue commented on a diff in pull request #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
baseline.gradle:
##########
@@ -57,8 +57,8 @@ subprojects {
   pluginManager.withPlugin('com.palantir.baseline-error-prone') {
     tasks.withType(JavaCompile).configureEach {
       options.errorprone.errorproneArgs.addAll (
-          // error-prone is slow, don't run on tests and generated src
-          '-XepExcludedPaths:.*/(test|generated-src)/.*',
+          // error-prone is slow, don't run on tests/generated-src/generated
+          '-XepExcludedPaths:.*/(test|generated-src|generated)/.*',

Review Comment:
   Can we use the same `generated-src` directory? Otherwise the generated classes don't show up in an IDE without reconfiguring. If not, we should discuss this on the dev list to make sure everyone knows it's going to cause trouble in IDEs.



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
versions.props:
##########
@@ -27,6 +27,7 @@ com.google.cloud:libraries-bom = 24.1.0
 org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0
 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0
 com.emc.ecs:object-client-bundle = 3.3.2
+org.immutables:value = 2.9.0

Review Comment:
   all dependencies that `immutables.value` requires are properly repackaged and the guava classes live under `org.immutables.value.internal.$guava$`. However, I didn't realize that they just released a new version, so I'm upgrading this to `2.9.2`



-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java:
##########
@@ -45,86 +45,54 @@ public void nullMetrics() {
   @Test
   public void missingFields() {
     Assertions.assertThat(ScanMetricsResultParser.fromJson("{}"))
-        .isEqualTo(new ScanMetricsResult(null, null, null, null, null, null, null, null, null));
-
-    TimerResult totalPlanningDuration = new TimerResult(TimeUnit.HOURS, Duration.ofHours(10), 3L);
-    CounterResult resultDataFiles = new CounterResult(Unit.COUNT, 5L);
-    CounterResult resultDeleteFiles = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalDeleteManifests = new CounterResult(Unit.COUNT, 0L);
-    CounterResult scannedDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult skippedDataManifests = new CounterResult(Unit.COUNT, 5L);
-    CounterResult totalFileSizeInBytes = new CounterResult(Unit.BYTES, 1069L);
+        .isEqualTo(ImmutableScanMetricsResult.builder().build());
 
+    ImmutableScanMetricsResult scanMetricsResult =

Review Comment:
   Oh, I see. 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 pull request #5780: API/Core: Make ScanReport and its related classes Immutable

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

   Thanks, @nastra! Looks great.


-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


-- 
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 #5780: API/Core: Make ScanReport and its related classes Immutable

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


##########
build.gradle:
##########
@@ -230,6 +230,8 @@ project(':iceberg-api') {
   dependencies {
     implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
     compileOnly "com.google.errorprone:error_prone_annotations"
+    annotationProcessor "org.immutables:value"
+    implementation "org.immutables:value"

Review Comment:
   technically it's a `compileOnly` dependency, I just forgot to set it properly to `compileOnly`



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