From a729952c4ba72e4e60cbe8b5c79675419040274e Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Fri, 21 Apr 2023 15:12:29 -0700 Subject: [PATCH 01/14] Show spark metrics on SQL UI --- .../spark/source/SparkBatchQueryScan.java | 35 ++++++++++- .../spark/source/SparkScanBuilder.java | 31 ++++++++-- .../metrics/SparkReadMetricReporter.java | 36 +++++++++++ .../source/metrics/TaskTotalFileSize.java | 40 +++++++++++++ .../metrics/TaskTotalPlanningDuration.java | 40 +++++++++++++ .../spark/source/metrics/TotalFileSize.java | 42 +++++++++++++ .../source/metrics/TotalPlanningDuration.java | 42 +++++++++++++ .../spark/source/TestSparkReadMetrics.java | 60 +++++++++++++++++++ 8 files changed, 318 insertions(+), 8 deletions(-) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index dd493fbc5097..cf248a881e24 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import org.apache.iceberg.PartitionField; @@ -39,6 +40,8 @@ import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -46,9 +49,16 @@ import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.source.metrics.SparkReadMetricReporter; +import org.apache.iceberg.spark.source.metrics.TaskTotalFileSize; +import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; +import org.apache.iceberg.spark.source.metrics.TotalFileSize; +import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering; import org.apache.spark.sql.sources.Filter; @@ -65,6 +75,7 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan private final Long endSnapshotId; private final Long asOfTimestamp; private final String tag; + private final SparkReadMetricReporter sparkReadMetricReporter; private final List runtimeFilterExpressions; SparkBatchQueryScan( @@ -73,8 +84,8 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan Scan> scan, SparkReadConf readConf, Schema expectedSchema, - List filters) { - + List filters, + SparkReadMetricReporter sparkReadMetricReporter) { super(spark, table, scan, readConf, expectedSchema, filters); this.snapshotId = readConf.snapshotId(); @@ -82,6 +93,7 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan this.endSnapshotId = readConf.endSnapshotId(); this.asOfTimestamp = readConf.asOfTimestamp(); this.tag = readConf.tag(); + this.sparkReadMetricReporter = sparkReadMetricReporter; this.runtimeFilterExpressions = Lists.newArrayList(); } @@ -256,4 +268,23 @@ public String toString() { runtimeFilterExpressions, caseSensitive()); } + + @Override + public CustomTaskMetric[] reportDriverMetrics() { + List customTaskMetrics = Lists.newArrayList(); + MetricsReport metricsReport = sparkReadMetricReporter.getMetricsReport(); + ScanReport scanReport = (ScanReport) metricsReport; + + Optional.ofNullable(scanReport.scanMetrics().totalFileSizeInBytes()) + .ifPresent(counterResult -> new TaskTotalFileSize(counterResult.value())); + Optional.ofNullable(scanReport.scanMetrics().totalPlanningDuration()) + .ifPresent(timerResult -> new TaskTotalPlanningDuration(timerResult.count())); + + return customTaskMetrics.toArray(new CustomTaskMetric[0]); + } + + @Override + public CustomMetric[] supportedCustomMetrics() { + return new CustomMetric[] {new TotalFileSize(), new TotalPlanningDuration()}; + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index ddeec9c4943b..27b497b4242f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -52,6 +52,7 @@ import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.source.metrics.SparkReadMetricReporter; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; @@ -96,6 +97,7 @@ public class SparkScanBuilder private boolean caseSensitive; private List filterExpressions = null; private Filter[] pushedFilters = NO_FILTERS; + private SparkReadMetricReporter sparkReadMetricReporter; SparkScanBuilder( SparkSession spark, @@ -109,6 +111,7 @@ public class SparkScanBuilder this.options = options; this.readConf = new SparkReadConf(spark, table, branch, options); this.caseSensitive = readConf.caseSensitive(); + this.sparkReadMetricReporter = new SparkReadMetricReporter(); } SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { @@ -425,12 +428,15 @@ private Scan buildBatchScan() { private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, String tag) { Schema expectedSchema = schemaWithMetadataColumns(); + sparkReadMetricReporter = new SparkReadMetricReporter(); + BatchScan scan = table .newBatchScan() .caseSensitive(caseSensitive) .filter(filterExpression()) - .project(expectedSchema); + .project(expectedSchema) + .metricsReporter(sparkReadMetricReporter); if (snapshotId != null) { scan = scan.useSnapshot(snapshotId); @@ -449,8 +455,8 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, } scan = configureSplitPlanning(scan); - - return new SparkBatchQueryScan(spark, table, scan, readConf, expectedSchema, filterExpressions); + return new SparkBatchQueryScan( + spark, table, scan, readConf, expectedSchema, filterExpressions, sparkReadMetricReporter); } private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId) { @@ -470,7 +476,8 @@ private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId scan = configureSplitPlanning(scan); - return new SparkBatchQueryScan(spark, table, scan, readConf, expectedSchema, filterExpressions); + return new SparkBatchQueryScan( + spark, table, scan, readConf, expectedSchema, filterExpressions, sparkReadMetricReporter); } public Scan buildChangelogScan() { @@ -573,7 +580,13 @@ public Scan buildMergeOnReadScan() { if (snapshot == null) { return new SparkBatchQueryScan( - spark, table, null, readConf, schemaWithMetadataColumns(), filterExpressions); + spark, + table, + null, + readConf, + schemaWithMetadataColumns(), + filterExpressions, + sparkReadMetricReporter); } // remember the current snapshot ID for commit validation @@ -597,7 +610,13 @@ public Scan buildMergeOnReadScan() { scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, table, scan, adjustedReadConf, expectedSchema, filterExpressions); + spark, + table, + scan, + adjustedReadConf, + expectedSchema, + filterExpressions, + sparkReadMetricReporter); } public Scan buildCopyOnWriteScan() { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java new file mode 100644 index 000000000000..8b19e19a460c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java @@ -0,0 +1,36 @@ +/* + * 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.spark.source.metrics; + +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.MetricsReporter; + +public class SparkReadMetricReporter implements MetricsReporter { + + private MetricsReport metricsReport; + + @Override + public void report(MetricsReport report) { + this.metricsReport = report; + } + + public MetricsReport getMetricsReport() { + return metricsReport; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java new file mode 100644 index 000000000000..99b1e8a12fab --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java @@ -0,0 +1,40 @@ +/* + * 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.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalFileSize implements CustomTaskMetric { + + private final long value; + + public TaskTotalFileSize(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalFileSize.name; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java new file mode 100644 index 000000000000..2751f0f99e85 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java @@ -0,0 +1,40 @@ +/* + * 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.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalPlanningDuration implements CustomTaskMetric { + + private final long value; + + public TaskTotalPlanningDuration(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalPlanningDuration.name; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java new file mode 100644 index 000000000000..230b0eff1c11 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java @@ -0,0 +1,42 @@ +/* + * 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.spark.source.metrics; + +import java.util.Arrays; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class TotalFileSize implements CustomMetric { + + public static final String name = "totalFileSize"; + + @Override + public String name() { + return name; + } + + @Override + public String description() { + return "Total File Size"; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + return String.valueOf(Arrays.stream(taskMetrics).sum()); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java new file mode 100644 index 000000000000..2ce0032c16eb --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -0,0 +1,42 @@ +/* + * 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.spark.source.metrics; + +import java.util.Arrays; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class TotalPlanningDuration implements CustomMetric { + + public static final String name = "planningDuration"; + + @Override + public String name() { + return name; + } + + @Override + public String description() { + return "Planning duration"; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + return String.valueOf(Arrays.stream(taskMetrics).sum()); + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java new file mode 100644 index 000000000000..e66b9c273f34 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -0,0 +1,60 @@ +/* + * 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.spark.source; + +import static scala.collection.JavaConverters.seqAsJavaListConverter; + +import java.util.List; +import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.source.metrics.TotalFileSize; +import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.metric.SQLMetric; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; +import scala.collection.immutable.Map; + +public class TestSparkReadMetrics extends SparkTestBaseWithCatalog { + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testReadMetrics() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT) USING iceberg", tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + + Dataset df = spark.sql(String.format("select * from %s", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metrics = sparkPlans.get(0).metrics(); + + Assert.assertTrue(metrics.contains(TotalFileSize.name)); + Assert.assertTrue(metrics.contains(TotalPlanningDuration.name)); + } +} From 14d7649b720098e7e841febf3331d5472d4e0183 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Thu, 4 May 2023 07:10:38 -0700 Subject: [PATCH 02/14] Add additional fixes, bug fixes --- .../spark/source/SparkBatchQueryScan.java | 42 +++++++++++++++++-- .../spark/source/metrics/ResultDataFiles.java | 42 +++++++++++++++++++ .../source/metrics/ScannedDataManifests.java | 42 +++++++++++++++++++ .../source/metrics/SkippedDataFiles.java | 42 +++++++++++++++++++ .../source/metrics/SkippedDataManifests.java | 42 +++++++++++++++++++ .../source/metrics/TaskResultDataFiles.java | 39 +++++++++++++++++ .../metrics/TaskScannedDataManifests.java | 39 +++++++++++++++++ .../source/metrics/TaskSkippedDataFiles.java | 39 +++++++++++++++++ .../metrics/TaskSkippedDataManifests.java | 39 +++++++++++++++++ .../spark/source/metrics/TotalFileSize.java | 2 +- .../source/metrics/TotalPlanningDuration.java | 2 +- .../spark/source/TestSparkReadMetrics.java | 12 +++--- 12 files changed, 370 insertions(+), 12 deletions(-) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index cf248a881e24..9a839a279cab 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -49,7 +49,15 @@ import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.source.metrics.ResultDataFiles; +import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; import org.apache.iceberg.spark.source.metrics.SparkReadMetricReporter; +import org.apache.iceberg.spark.source.metrics.TaskResultDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataManifests; import org.apache.iceberg.spark.source.metrics.TaskTotalFileSize; import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; import org.apache.iceberg.spark.source.metrics.TotalFileSize; @@ -276,15 +284,43 @@ public CustomTaskMetric[] reportDriverMetrics() { ScanReport scanReport = (ScanReport) metricsReport; Optional.ofNullable(scanReport.scanMetrics().totalFileSizeInBytes()) - .ifPresent(counterResult -> new TaskTotalFileSize(counterResult.value())); + .ifPresent( + counterResult -> customTaskMetrics.add(new TaskTotalFileSize(counterResult.value()))); Optional.ofNullable(scanReport.scanMetrics().totalPlanningDuration()) - .ifPresent(timerResult -> new TaskTotalPlanningDuration(timerResult.count())); + .ifPresent( + timerResult -> + customTaskMetrics.add(new TaskTotalPlanningDuration(timerResult.count()))); + + Optional.ofNullable(scanReport.scanMetrics().skippedDataFiles()) + .ifPresent( + skippedDataFilesResult -> + customTaskMetrics.add(new TaskSkippedDataFiles(skippedDataFilesResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().resultDataFiles()) + .ifPresent(resultDataFilesResult -> new TaskResultDataFiles(resultDataFilesResult.value())); + + Optional.ofNullable(scanReport.scanMetrics().skippedDataManifests()) + .ifPresent( + skippedDataManifestsResult -> + customTaskMetrics.add( + new TaskSkippedDataManifests(skippedDataManifestsResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().scannedDataManifests()) + .ifPresent( + scannedDataManifestResult -> + customTaskMetrics.add( + new TaskScannedDataManifests(scannedDataManifestResult.value()))); return customTaskMetrics.toArray(new CustomTaskMetric[0]); } @Override public CustomMetric[] supportedCustomMetrics() { - return new CustomMetric[] {new TotalFileSize(), new TotalPlanningDuration()}; + return new CustomMetric[] { + new TotalFileSize(), + new TotalPlanningDuration(), + new ScannedDataManifests(), + new SkippedDataManifests(), + new ResultDataFiles(), + new SkippedDataFiles() + }; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java new file mode 100644 index 000000000000..35af0509f753 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java @@ -0,0 +1,42 @@ +/* + * 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.spark.source.metrics; + +import java.util.Arrays; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class ResultDataFiles implements CustomMetric { + + static final String name = "resultDataFiles"; + + @Override + public String name() { + return name; + } + + @Override + public String description() { + return "Result data files"; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + return String.valueOf(Arrays.stream(taskMetrics).sum()); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java new file mode 100644 index 000000000000..646b8ed47145 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -0,0 +1,42 @@ +/* + * 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.spark.source.metrics; + +import java.util.Arrays; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class ScannedDataManifests implements CustomMetric { + + static final String name = "scannedDataManifests"; + + @Override + public String name() { + return name; + } + + @Override + public String description() { + return "Num scanned data manifests"; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + return String.valueOf(Arrays.stream(taskMetrics).sum()); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java new file mode 100644 index 000000000000..6b309f8cf810 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java @@ -0,0 +1,42 @@ +/* + * 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.spark.source.metrics; + +import java.util.Arrays; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class SkippedDataFiles implements CustomMetric { + + static final String name = "skippedDataFiles"; + + @Override + public String name() { + return name; + } + + @Override + public String description() { + return "Num skipped data files"; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + return String.valueOf(Arrays.stream(taskMetrics).sum()); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java new file mode 100644 index 000000000000..637d31a2a4b1 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java @@ -0,0 +1,42 @@ +/* + * 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.spark.source.metrics; + +import java.util.Arrays; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class SkippedDataManifests implements CustomMetric { + + static final String name = "skippedDataManifests"; + + @Override + public String name() { + return name; + } + + @Override + public String description() { + return "Num skipped data manifests"; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + return String.valueOf(Arrays.stream(taskMetrics).sum()); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java new file mode 100644 index 000000000000..3c80fa5435a6 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java @@ -0,0 +1,39 @@ +/* + * 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.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskResultDataFiles implements CustomTaskMetric { + private final long value; + + public TaskResultDataFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return ResultDataFiles.name; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java new file mode 100644 index 000000000000..f400f57d7868 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java @@ -0,0 +1,39 @@ +/* + * 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.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskScannedDataManifests implements CustomTaskMetric { + private final long value; + + public TaskScannedDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return ScannedDataManifests.name; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java new file mode 100644 index 000000000000..f06d28eb45cf --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java @@ -0,0 +1,39 @@ +/* + * 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.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDataFiles implements CustomTaskMetric { + private final long value; + + public TaskSkippedDataFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDataFiles.name; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java new file mode 100644 index 000000000000..1168967bbbe2 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java @@ -0,0 +1,39 @@ +/* + * 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.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDataManifests implements CustomTaskMetric { + private final long value; + + public TaskSkippedDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDataManifests.name; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java index 230b0eff1c11..1d2336181cbd 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java @@ -23,7 +23,7 @@ public class TotalFileSize implements CustomMetric { - public static final String name = "totalFileSize"; + static final String name = "totalFileSize"; @Override public String name() { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java index 2ce0032c16eb..7f02d2bbc952 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -23,7 +23,7 @@ public class TotalPlanningDuration implements CustomMetric { - public static final String name = "planningDuration"; + static final String name = "planningDuration"; @Override public String name() { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index e66b9c273f34..02a04d97122b 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -21,9 +21,8 @@ import static scala.collection.JavaConverters.seqAsJavaListConverter; import java.util.List; +import java.util.Map; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; -import org.apache.iceberg.spark.source.metrics.TotalFileSize; -import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; @@ -32,7 +31,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Test; -import scala.collection.immutable.Map; +import scala.collection.JavaConverters; public class TestSparkReadMetrics extends SparkTestBaseWithCatalog { @@ -52,9 +51,8 @@ public void testReadMetrics() throws NoSuchTableException { List sparkPlans = seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); - Map metrics = sparkPlans.get(0).metrics(); - - Assert.assertTrue(metrics.contains(TotalFileSize.name)); - Assert.assertTrue(metrics.contains(TotalPlanningDuration.name)); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + Assert.assertEquals(1, metricsMap.get("scannedDataManifests").value()); } } From b3ec3896e9e4f8967ffb15daf8c0e78d60e58cb6 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Sun, 7 May 2023 15:08:35 -0700 Subject: [PATCH 03/14] Address review comments --- .../spark/source/SparkBatchQueryScan.java | 58 +++++++++---------- .../metrics/SparkReadMetricReporter.java | 6 +- .../spark/source/TestSparkReadMetrics.java | 4 +- 3 files changed, 35 insertions(+), 33 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index 9a839a279cab..d7d6139ab30e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -40,7 +40,6 @@ import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Projections; -import org.apache.iceberg.metrics.MetricsReport; import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -280,34 +279,35 @@ public String toString() { @Override public CustomTaskMetric[] reportDriverMetrics() { List customTaskMetrics = Lists.newArrayList(); - MetricsReport metricsReport = sparkReadMetricReporter.getMetricsReport(); - ScanReport scanReport = (ScanReport) metricsReport; - - Optional.ofNullable(scanReport.scanMetrics().totalFileSizeInBytes()) - .ifPresent( - counterResult -> customTaskMetrics.add(new TaskTotalFileSize(counterResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().totalPlanningDuration()) - .ifPresent( - timerResult -> - customTaskMetrics.add(new TaskTotalPlanningDuration(timerResult.count()))); - - Optional.ofNullable(scanReport.scanMetrics().skippedDataFiles()) - .ifPresent( - skippedDataFilesResult -> - customTaskMetrics.add(new TaskSkippedDataFiles(skippedDataFilesResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().resultDataFiles()) - .ifPresent(resultDataFilesResult -> new TaskResultDataFiles(resultDataFilesResult.value())); - - Optional.ofNullable(scanReport.scanMetrics().skippedDataManifests()) - .ifPresent( - skippedDataManifestsResult -> - customTaskMetrics.add( - new TaskSkippedDataManifests(skippedDataManifestsResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().scannedDataManifests()) - .ifPresent( - scannedDataManifestResult -> - customTaskMetrics.add( - new TaskScannedDataManifests(scannedDataManifestResult.value()))); + Optional scanReportOptional = sparkReadMetricReporter.getScanReport(); + + scanReportOptional.ifPresent(scanReport -> { + Optional.ofNullable(scanReport.scanMetrics().totalFileSizeInBytes()) + .ifPresent( + counterResult -> customTaskMetrics.add(new TaskTotalFileSize(counterResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().totalPlanningDuration()) + .ifPresent( + timerResult -> + customTaskMetrics.add(new TaskTotalPlanningDuration(timerResult.count()))); + + Optional.ofNullable(scanReport.scanMetrics().skippedDataFiles()) + .ifPresent( + skippedDataFilesResult -> + customTaskMetrics.add(new TaskSkippedDataFiles(skippedDataFilesResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().resultDataFiles()) + .ifPresent(resultDataFilesResult -> new TaskResultDataFiles(resultDataFilesResult.value())); + + Optional.ofNullable(scanReport.scanMetrics().skippedDataManifests()) + .ifPresent( + skippedDataManifestsResult -> + customTaskMetrics.add( + new TaskSkippedDataManifests(skippedDataManifestsResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().scannedDataManifests()) + .ifPresent( + scannedDataManifestResult -> + customTaskMetrics.add( + new TaskScannedDataManifests(scannedDataManifestResult.value()))); + }); return customTaskMetrics.toArray(new CustomTaskMetric[0]); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java index 8b19e19a460c..e59c2fa9bd2b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.spark.source.metrics; +import java.util.Optional; import org.apache.iceberg.metrics.MetricsReport; import org.apache.iceberg.metrics.MetricsReporter; +import org.apache.iceberg.metrics.ScanReport; public class SparkReadMetricReporter implements MetricsReporter { @@ -30,7 +32,7 @@ public void report(MetricsReport report) { this.metricsReport = report; } - public MetricsReport getMetricsReport() { - return metricsReport; + public Optional getScanReport() { + return Optional.ofNullable((ScanReport) metricsReport); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index 02a04d97122b..782eb9697e9a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -29,8 +29,8 @@ import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.metric.SQLMetric; import org.junit.After; -import org.junit.Assert; import org.junit.Test; +import org.junit.jupiter.api.Assertions; import scala.collection.JavaConverters; public class TestSparkReadMetrics extends SparkTestBaseWithCatalog { @@ -53,6 +53,6 @@ public void testReadMetrics() throws NoSuchTableException { seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); Map metricsMap = JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); - Assert.assertEquals(1, metricsMap.get("scannedDataManifests").value()); + Assertions.assertEquals(1, metricsMap.get("scannedDataManifests").value()); } } From 4663cc29f8984b9246bbaaf62755896c000c6d97 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Mon, 8 May 2023 05:49:55 -0700 Subject: [PATCH 04/14] Fix spotless error --- .../spark/source/SparkBatchQueryScan.java | 58 ++++++++++--------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index d7d6139ab30e..54c39bc8a100 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -281,33 +281,37 @@ public CustomTaskMetric[] reportDriverMetrics() { List customTaskMetrics = Lists.newArrayList(); Optional scanReportOptional = sparkReadMetricReporter.getScanReport(); - scanReportOptional.ifPresent(scanReport -> { - Optional.ofNullable(scanReport.scanMetrics().totalFileSizeInBytes()) - .ifPresent( - counterResult -> customTaskMetrics.add(new TaskTotalFileSize(counterResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().totalPlanningDuration()) - .ifPresent( - timerResult -> - customTaskMetrics.add(new TaskTotalPlanningDuration(timerResult.count()))); - - Optional.ofNullable(scanReport.scanMetrics().skippedDataFiles()) - .ifPresent( - skippedDataFilesResult -> - customTaskMetrics.add(new TaskSkippedDataFiles(skippedDataFilesResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().resultDataFiles()) - .ifPresent(resultDataFilesResult -> new TaskResultDataFiles(resultDataFilesResult.value())); - - Optional.ofNullable(scanReport.scanMetrics().skippedDataManifests()) - .ifPresent( - skippedDataManifestsResult -> - customTaskMetrics.add( - new TaskSkippedDataManifests(skippedDataManifestsResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().scannedDataManifests()) - .ifPresent( - scannedDataManifestResult -> - customTaskMetrics.add( - new TaskScannedDataManifests(scannedDataManifestResult.value()))); - }); + scanReportOptional.ifPresent( + scanReport -> { + Optional.ofNullable(scanReport.scanMetrics().totalFileSizeInBytes()) + .ifPresent( + counterResult -> + customTaskMetrics.add(new TaskTotalFileSize(counterResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().totalPlanningDuration()) + .ifPresent( + timerResult -> + customTaskMetrics.add(new TaskTotalPlanningDuration(timerResult.count()))); + + Optional.ofNullable(scanReport.scanMetrics().skippedDataFiles()) + .ifPresent( + skippedDataFilesResult -> + customTaskMetrics.add( + new TaskSkippedDataFiles(skippedDataFilesResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().resultDataFiles()) + .ifPresent( + resultDataFilesResult -> new TaskResultDataFiles(resultDataFilesResult.value())); + + Optional.ofNullable(scanReport.scanMetrics().skippedDataManifests()) + .ifPresent( + skippedDataManifestsResult -> + customTaskMetrics.add( + new TaskSkippedDataManifests(skippedDataManifestsResult.value()))); + Optional.ofNullable(scanReport.scanMetrics().scannedDataManifests()) + .ifPresent( + scannedDataManifestResult -> + customTaskMetrics.add( + new TaskScannedDataManifests(scannedDataManifestResult.value()))); + }); return customTaskMetrics.toArray(new CustomTaskMetric[0]); } From 13547ea660854db2ab6c08a2ca00e3517244fc93 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Mon, 26 Jun 2023 07:27:43 -0700 Subject: [PATCH 05/14] Address review comments --- .../spark/source/SparkBatchQueryScan.java | 75 +------------------ .../spark/source/SparkCopyOnWriteScan.java | 12 ++- .../source/SparkPartitioningAwareScan.java | 7 +- .../iceberg/spark/source/SparkScan.java | 51 ++++++++++++- .../spark/source/SparkScanBuilder.java | 28 +++---- .../iceberg/spark/source/SparkStagedScan.java | 10 ++- .../spark/source/SparkStagedScanBuilder.java | 3 +- ...r.java => InMemoryReadMetricReporter.java} | 9 ++- .../spark/source/metrics/ResultDataFiles.java | 12 +-- .../source/metrics/ScannedDataManifests.java | 12 +-- .../source/metrics/SkippedDataFiles.java | 12 +-- .../source/metrics/SkippedDataManifests.java | 12 +-- .../source/metrics/TaskResultDataFiles.java | 8 ++ .../metrics/TaskScannedDataManifests.java | 8 ++ .../source/metrics/TaskSkippedDataFiles.java | 8 ++ .../metrics/TaskSkippedDataManifests.java | 8 ++ .../source/metrics/TaskTotalFileSize.java | 8 ++ .../metrics/TaskTotalPlanningDuration.java | 8 ++ 18 files changed, 150 insertions(+), 141 deletions(-) rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{SparkReadMetricReporter.java => InMemoryReadMetricReporter.java} (84%) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index 54c39bc8a100..6f3622a7cfee 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -21,8 +21,8 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionScanTask; @@ -48,24 +48,9 @@ import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.source.metrics.ResultDataFiles; -import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; -import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; -import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; -import org.apache.iceberg.spark.source.metrics.SparkReadMetricReporter; -import org.apache.iceberg.spark.source.metrics.TaskResultDataFiles; -import org.apache.iceberg.spark.source.metrics.TaskScannedDataManifests; -import org.apache.iceberg.spark.source.metrics.TaskSkippedDataFiles; -import org.apache.iceberg.spark.source.metrics.TaskSkippedDataManifests; -import org.apache.iceberg.spark.source.metrics.TaskTotalFileSize; -import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; -import org.apache.iceberg.spark.source.metrics.TotalFileSize; -import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.expressions.NamedReference; -import org.apache.spark.sql.connector.metric.CustomMetric; -import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering; import org.apache.spark.sql.sources.Filter; @@ -82,7 +67,6 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan private final Long endSnapshotId; private final Long asOfTimestamp; private final String tag; - private final SparkReadMetricReporter sparkReadMetricReporter; private final List runtimeFilterExpressions; SparkBatchQueryScan( @@ -92,15 +76,14 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan SparkReadConf readConf, Schema expectedSchema, List filters, - SparkReadMetricReporter sparkReadMetricReporter) { - super(spark, table, scan, readConf, expectedSchema, filters); + Supplier metricsReportSupplier) { + super(spark, table, scan, readConf, expectedSchema, filters, metricsReportSupplier); this.snapshotId = readConf.snapshotId(); this.startSnapshotId = readConf.startSnapshotId(); this.endSnapshotId = readConf.endSnapshotId(); this.asOfTimestamp = readConf.asOfTimestamp(); this.tag = readConf.tag(); - this.sparkReadMetricReporter = sparkReadMetricReporter; this.runtimeFilterExpressions = Lists.newArrayList(); } @@ -275,56 +258,4 @@ public String toString() { runtimeFilterExpressions, caseSensitive()); } - - @Override - public CustomTaskMetric[] reportDriverMetrics() { - List customTaskMetrics = Lists.newArrayList(); - Optional scanReportOptional = sparkReadMetricReporter.getScanReport(); - - scanReportOptional.ifPresent( - scanReport -> { - Optional.ofNullable(scanReport.scanMetrics().totalFileSizeInBytes()) - .ifPresent( - counterResult -> - customTaskMetrics.add(new TaskTotalFileSize(counterResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().totalPlanningDuration()) - .ifPresent( - timerResult -> - customTaskMetrics.add(new TaskTotalPlanningDuration(timerResult.count()))); - - Optional.ofNullable(scanReport.scanMetrics().skippedDataFiles()) - .ifPresent( - skippedDataFilesResult -> - customTaskMetrics.add( - new TaskSkippedDataFiles(skippedDataFilesResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().resultDataFiles()) - .ifPresent( - resultDataFilesResult -> new TaskResultDataFiles(resultDataFilesResult.value())); - - Optional.ofNullable(scanReport.scanMetrics().skippedDataManifests()) - .ifPresent( - skippedDataManifestsResult -> - customTaskMetrics.add( - new TaskSkippedDataManifests(skippedDataManifestsResult.value()))); - Optional.ofNullable(scanReport.scanMetrics().scannedDataManifests()) - .ifPresent( - scannedDataManifestResult -> - customTaskMetrics.add( - new TaskScannedDataManifests(scannedDataManifestResult.value()))); - }); - - return customTaskMetrics.toArray(new CustomTaskMetric[0]); - } - - @Override - public CustomMetric[] supportedCustomMetrics() { - return new CustomMetric[] { - new TotalFileSize(), - new TotalPlanningDuration(), - new ScannedDataManifests(), - new SkippedDataManifests(), - new ResultDataFiles(), - new SkippedDataFiles() - }; - } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index d978b81e67bd..e32b68fb054f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.iceberg.BatchScan; import org.apache.iceberg.FileScanTask; @@ -30,6 +31,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkReadConf; @@ -57,8 +59,9 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan Table table, SparkReadConf readConf, Schema expectedSchema, - List filters) { - this(spark, table, null, null, readConf, expectedSchema, filters); + List filters, + Supplier scanReportSupplier) { + this(spark, table, null, null, readConf, expectedSchema, filters, scanReportSupplier); } SparkCopyOnWriteScan( @@ -68,9 +71,10 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan Snapshot snapshot, SparkReadConf readConf, Schema expectedSchema, - List filters) { + List filters, + Supplier scanReportSupplier) { - super(spark, table, scan, readConf, expectedSchema, filters); + super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); this.snapshot = snapshot; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java index cf274f794e52..eae901ff1caa 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.iceberg.PartitionField; @@ -37,6 +38,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; @@ -74,9 +76,10 @@ abstract class SparkPartitioningAwareScan extends S Scan> scan, SparkReadConf readConf, Schema expectedSchema, - List filters) { + List filters, + Supplier metricsReportSupplier) { - super(spark, table, readConf, expectedSchema, filters); + super(spark, table, readConf, expectedSchema, filters, metricsReportSupplier); this.scan = scan; this.preserveDataGrouping = readConf.preserveDataGrouping(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index b47818ec550d..8974be7f36b0 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; @@ -28,17 +29,33 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.source.metrics.NumDeletes; import org.apache.iceberg.spark.source.metrics.NumSplits; +import org.apache.iceberg.spark.source.metrics.ResultDataFiles; +import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskResultDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskTotalFileSize; +import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; +import org.apache.iceberg.spark.source.metrics.TotalFileSize; +import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.Statistics; @@ -58,6 +75,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { private final Schema expectedSchema; private final List filterExpressions; private final String branch; + private final Supplier metricsReportSupplier; // lazy variables private StructType readSchema; @@ -67,7 +85,9 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { Table table, SparkReadConf readConf, Schema expectedSchema, - List filters) { + List filters, + Supplier metricsReportSupplier) { + this.metricsReportSupplier = metricsReportSupplier; Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); @@ -170,8 +190,35 @@ public String description() { table(), branch(), Spark3Util.describe(filterExpressions), groupingKeyFieldNamesAsString); } + @Override + public CustomTaskMetric[] reportDriverMetrics() { + ScanReport scanReport = metricsReportSupplier.get(); + if (scanReport == null) { + return new CustomTaskMetric[0]; + } + + List driverMetrics = Lists.newArrayList(); + driverMetrics.add(TaskTotalFileSize.from(scanReport)); + driverMetrics.add(TaskTotalPlanningDuration.from(scanReport)); + driverMetrics.add(TaskSkippedDataFiles.from(scanReport)); + driverMetrics.add(TaskResultDataFiles.from(scanReport)); + driverMetrics.add(TaskSkippedDataManifests.from(scanReport)); + driverMetrics.add(TaskScannedDataManifests.from(scanReport)); + + return driverMetrics.toArray(new CustomTaskMetric[0]); + } + @Override public CustomMetric[] supportedCustomMetrics() { - return new CustomMetric[] {new NumSplits(), new NumDeletes()}; + return new CustomMetric[] { + new NumSplits(), + new NumDeletes(), + new TotalFileSize(), + new TotalPlanningDuration(), + new ScannedDataManifests(), + new SkippedDataManifests(), + new ResultDataFiles(), + new SkippedDataFiles() + }; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 27b497b4242f..7fd92310a77f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -52,7 +52,7 @@ import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.source.metrics.SparkReadMetricReporter; +import org.apache.iceberg.spark.source.metrics.InMemoryReadMetricReporter; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; @@ -97,7 +97,7 @@ public class SparkScanBuilder private boolean caseSensitive; private List filterExpressions = null; private Filter[] pushedFilters = NO_FILTERS; - private SparkReadMetricReporter sparkReadMetricReporter; + private final InMemoryReadMetricReporter metricsReporter; SparkScanBuilder( SparkSession spark, @@ -111,7 +111,7 @@ public class SparkScanBuilder this.options = options; this.readConf = new SparkReadConf(spark, table, branch, options); this.caseSensitive = readConf.caseSensitive(); - this.sparkReadMetricReporter = new SparkReadMetricReporter(); + this.metricsReporter = new InMemoryReadMetricReporter(); } SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { @@ -428,15 +428,13 @@ private Scan buildBatchScan() { private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, String tag) { Schema expectedSchema = schemaWithMetadataColumns(); - sparkReadMetricReporter = new SparkReadMetricReporter(); - BatchScan scan = table .newBatchScan() .caseSensitive(caseSensitive) .filter(filterExpression()) .project(expectedSchema) - .metricsReporter(sparkReadMetricReporter); + .metricsReporter(metricsReporter); if (snapshotId != null) { scan = scan.useSnapshot(snapshotId); @@ -456,7 +454,7 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, table, scan, readConf, expectedSchema, filterExpressions, sparkReadMetricReporter); + spark, table, scan, readConf, expectedSchema, filterExpressions, metricsReporter); } private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId) { @@ -477,7 +475,7 @@ private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, table, scan, readConf, expectedSchema, filterExpressions, sparkReadMetricReporter); + spark, table, scan, readConf, expectedSchema, filterExpressions, metricsReporter); } public Scan buildChangelogScan() { @@ -586,7 +584,7 @@ public Scan buildMergeOnReadScan() { readConf, schemaWithMetadataColumns(), filterExpressions, - sparkReadMetricReporter); + metricsReporter); } // remember the current snapshot ID for commit validation @@ -610,13 +608,7 @@ public Scan buildMergeOnReadScan() { scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, - table, - scan, - adjustedReadConf, - expectedSchema, - filterExpressions, - sparkReadMetricReporter); + spark, table, scan, adjustedReadConf, expectedSchema, filterExpressions, metricsReporter); } public Scan buildCopyOnWriteScan() { @@ -624,7 +616,7 @@ public Scan buildCopyOnWriteScan() { if (snapshot == null) { return new SparkCopyOnWriteScan( - spark, table, readConf, schemaWithMetadataColumns(), filterExpressions); + spark, table, readConf, schemaWithMetadataColumns(), filterExpressions, metricsReporter); } Schema expectedSchema = schemaWithMetadataColumns(); @@ -641,7 +633,7 @@ public Scan buildCopyOnWriteScan() { scan = configureSplitPlanning(scan); return new SparkCopyOnWriteScan( - spark, table, scan, snapshot, readConf, expectedSchema, filterExpressions); + spark, table, scan, snapshot, readConf, expectedSchema, filterExpressions, metricsReporter); } private > T configureSplitPlanning(T scan) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java index 89b184c91c51..a0f5a954637c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -20,10 +20,12 @@ import java.util.List; import java.util.Objects; +import java.util.function.Supplier; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkReadConf; @@ -39,8 +41,12 @@ class SparkStagedScan extends SparkScan { private List> taskGroups = null; // lazy cache of tasks - SparkStagedScan(SparkSession spark, Table table, SparkReadConf readConf) { - super(spark, table, readConf, table.schema(), ImmutableList.of()); + SparkStagedScan( + SparkSession spark, + Table table, + SparkReadConf readConf, + Supplier scanReportSupplier) { + super(spark, table, readConf, table.schema(), ImmutableList.of(), scanReportSupplier); this.taskSetId = readConf.scanTaskSetId(); this.splitSize = readConf.splitSize(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 37bbea42e5b1..a5e9e1216b02 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -20,6 +20,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.source.metrics.InMemoryReadMetricReporter; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; @@ -39,6 +40,6 @@ class SparkStagedScanBuilder implements ScanBuilder { @Override public Scan build() { - return new SparkStagedScan(spark, table, readConf); + return new SparkStagedScan(spark, table, readConf, new InMemoryReadMetricReporter()); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java similarity index 84% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java index e59c2fa9bd2b..ed60b951f751 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SparkReadMetricReporter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java @@ -18,12 +18,12 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.Optional; +import java.util.function.Supplier; import org.apache.iceberg.metrics.MetricsReport; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.ScanReport; -public class SparkReadMetricReporter implements MetricsReporter { +public class InMemoryReadMetricReporter implements MetricsReporter, Supplier { private MetricsReport metricsReport; @@ -32,7 +32,8 @@ public void report(MetricsReport report) { this.metricsReport = report; } - public Optional getScanReport() { - return Optional.ofNullable((ScanReport) metricsReport); + @Override + public ScanReport get() { + return (ScanReport) metricsReport; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java index 35af0509f753..dccf9010de1c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.Arrays; -import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class ResultDataFiles implements CustomMetric { +public class ResultDataFiles extends CustomSumMetric { static final String name = "resultDataFiles"; @@ -32,11 +31,6 @@ public String name() { @Override public String description() { - return "Result data files"; - } - - @Override - public String aggregateTaskMetrics(long[] taskMetrics) { - return String.valueOf(Arrays.stream(taskMetrics).sum()); + return "result data files"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java index 646b8ed47145..00421c5693ae 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.Arrays; -import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class ScannedDataManifests implements CustomMetric { +public class ScannedDataManifests extends CustomSumMetric { static final String name = "scannedDataManifests"; @@ -32,11 +31,6 @@ public String name() { @Override public String description() { - return "Num scanned data manifests"; - } - - @Override - public String aggregateTaskMetrics(long[] taskMetrics) { - return String.valueOf(Arrays.stream(taskMetrics).sum()); + return "num scanned data manifests"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java index 6b309f8cf810..7f01cf16f8f4 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.Arrays; -import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class SkippedDataFiles implements CustomMetric { +public class SkippedDataFiles extends CustomSumMetric { static final String name = "skippedDataFiles"; @@ -32,11 +31,6 @@ public String name() { @Override public String description() { - return "Num skipped data files"; - } - - @Override - public String aggregateTaskMetrics(long[] taskMetrics) { - return String.valueOf(Arrays.stream(taskMetrics).sum()); + return "num skipped data files"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java index 637d31a2a4b1..bfef9988c3d8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.Arrays; -import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class SkippedDataManifests implements CustomMetric { +public class SkippedDataManifests extends CustomSumMetric { static final String name = "skippedDataManifests"; @@ -32,11 +31,6 @@ public String name() { @Override public String description() { - return "Num skipped data manifests"; - } - - @Override - public String aggregateTaskMetrics(long[] taskMetrics) { - return String.valueOf(Arrays.stream(taskMetrics).sum()); + return "num skipped data manifests"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java index 3c80fa5435a6..c69fa59b6445 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source.metrics; +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; public class TaskResultDataFiles implements CustomTaskMetric { @@ -36,4 +38,10 @@ public String name() { public long value() { return value; } + + public static TaskResultDataFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().resultDataFiles(); + long value = counter != null ? counter.value() : -1; + return new TaskResultDataFiles(value); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java index f400f57d7868..b9ed9e2d39c5 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source.metrics; +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; public class TaskScannedDataManifests implements CustomTaskMetric { @@ -36,4 +38,10 @@ public String name() { public long value() { return value; } + + public static TaskScannedDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().scannedDataManifests(); + long value = counter != null ? counter.value() : -1; + return new TaskScannedDataManifests(value); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java index f06d28eb45cf..9c08964c6da2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source.metrics; +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; public class TaskSkippedDataFiles implements CustomTaskMetric { @@ -36,4 +38,10 @@ public String name() { public long value() { return value; } + + public static TaskSkippedDataFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDataFiles(); + long value = counter != null ? counter.value() : -1; + return new TaskSkippedDataFiles(value); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java index 1168967bbbe2..63d28bf76cd9 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source.metrics; +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; public class TaskSkippedDataManifests implements CustomTaskMetric { @@ -36,4 +38,10 @@ public String name() { public long value() { return value; } + + public static TaskSkippedDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDataManifests(); + long value = counter != null ? counter.value() : -1; + return new TaskSkippedDataManifests(value); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java index 99b1e8a12fab..df8fd66e1fda 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source.metrics; +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; public class TaskTotalFileSize implements CustomTaskMetric { @@ -37,4 +39,10 @@ public String name() { public long value() { return value; } + + public static TaskTotalFileSize from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalFileSizeInBytes(); + long value = counter != null ? counter.value() : -1; + return new TaskTotalFileSize(value); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java index 2751f0f99e85..b7b60b7e21d1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source.metrics; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.metrics.TimerResult; import org.apache.spark.sql.connector.metric.CustomTaskMetric; public class TaskTotalPlanningDuration implements CustomTaskMetric { @@ -37,4 +39,10 @@ public String name() { public long value() { return value; } + + public static TaskTotalFileSize from(ScanReport scanReport) { + TimerResult timerResult = scanReport.scanMetrics().totalPlanningDuration(); + long value = timerResult != null ? timerResult.count() : -1; + return new TaskTotalFileSize(value); + } } From 7a113cd744d6a30d3026b28bd1248085561a46a8 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Mon, 26 Jun 2023 15:53:18 -0700 Subject: [PATCH 06/14] Address review comments --- .../iceberg/spark/source/SparkScan.java | 2 +- .../spark/source/SparkScanBuilder.java | 42 ++++++++++++++++--- .../spark/source/SparkStagedScanBuilder.java | 3 +- .../metrics/InMemoryReadMetricReporter.java | 14 +++---- .../spark/source/metrics/TotalFileSize.java | 2 +- .../source/metrics/TotalPlanningDuration.java | 4 +- 6 files changed, 49 insertions(+), 18 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 8974be7f36b0..44491c3e7d38 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -192,7 +192,7 @@ public String description() { @Override public CustomTaskMetric[] reportDriverMetrics() { - ScanReport scanReport = metricsReportSupplier.get(); + ScanReport scanReport = metricsReportSupplier != null ? metricsReportSupplier.get() : null; if (scanReport == null) { return new CustomTaskMetric[0]; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 7fd92310a77f..5a0057d7d14a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -454,7 +454,13 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, table, scan, readConf, expectedSchema, filterExpressions, metricsReporter); + spark, + table, + scan, + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId) { @@ -475,7 +481,13 @@ private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, table, scan, readConf, expectedSchema, filterExpressions, metricsReporter); + spark, + table, + scan, + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } public Scan buildChangelogScan() { @@ -584,7 +596,7 @@ public Scan buildMergeOnReadScan() { readConf, schemaWithMetadataColumns(), filterExpressions, - metricsReporter); + metricsReporter::scanReport); } // remember the current snapshot ID for commit validation @@ -608,7 +620,13 @@ public Scan buildMergeOnReadScan() { scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, table, scan, adjustedReadConf, expectedSchema, filterExpressions, metricsReporter); + spark, + table, + scan, + adjustedReadConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } public Scan buildCopyOnWriteScan() { @@ -616,7 +634,12 @@ public Scan buildCopyOnWriteScan() { if (snapshot == null) { return new SparkCopyOnWriteScan( - spark, table, readConf, schemaWithMetadataColumns(), filterExpressions, metricsReporter); + spark, + table, + readConf, + schemaWithMetadataColumns(), + filterExpressions, + metricsReporter::scanReport); } Schema expectedSchema = schemaWithMetadataColumns(); @@ -633,7 +656,14 @@ public Scan buildCopyOnWriteScan() { scan = configureSplitPlanning(scan); return new SparkCopyOnWriteScan( - spark, table, scan, snapshot, readConf, expectedSchema, filterExpressions, metricsReporter); + spark, + table, + scan, + snapshot, + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } private > T configureSplitPlanning(T scan) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index a5e9e1216b02..99e28724439b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -40,6 +40,7 @@ class SparkStagedScanBuilder implements ScanBuilder { @Override public Scan build() { - return new SparkStagedScan(spark, table, readConf, new InMemoryReadMetricReporter()); + return new SparkStagedScan( + spark, table, readConf, (new InMemoryReadMetricReporter())::scanReport); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java index ed60b951f751..079e205dae37 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java @@ -18,22 +18,22 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.function.Supplier; import org.apache.iceberg.metrics.MetricsReport; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.ScanReport; -public class InMemoryReadMetricReporter implements MetricsReporter, Supplier { +public class InMemoryReadMetricReporter implements MetricsReporter { - private MetricsReport metricsReport; + private ScanReport metricsReport; @Override public void report(MetricsReport report) { - this.metricsReport = report; + if (report instanceof ScanReport) { + this.metricsReport = (ScanReport) report; + } } - @Override - public ScanReport get() { - return (ScanReport) metricsReport; + public ScanReport scanReport() { + return metricsReport; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java index 1d2336181cbd..84f918f91691 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java @@ -32,7 +32,7 @@ public String name() { @Override public String description() { - return "Total File Size"; + return "total file size"; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java index 7f02d2bbc952..a4249e2e7fce 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -23,7 +23,7 @@ public class TotalPlanningDuration implements CustomMetric { - static final String name = "planningDuration"; + static final String name = "totalPlanningDuration"; @Override public String name() { @@ -32,7 +32,7 @@ public String name() { @Override public String description() { - return "Planning duration"; + return "total planning duration"; } @Override From 2735506cbc82ce81f446041bfe7d6018ed9b2b70 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Mon, 26 Jun 2023 15:56:08 -0700 Subject: [PATCH 07/14] Rename --- .../apache/iceberg/spark/source/metrics/ResultDataFiles.java | 4 ++-- .../iceberg/spark/source/metrics/ScannedDataManifests.java | 4 ++-- .../apache/iceberg/spark/source/metrics/SkippedDataFiles.java | 4 ++-- .../iceberg/spark/source/metrics/SkippedDataManifests.java | 4 ++-- .../iceberg/spark/source/metrics/TaskResultDataFiles.java | 2 +- .../spark/source/metrics/TaskScannedDataManifests.java | 2 +- .../iceberg/spark/source/metrics/TaskSkippedDataFiles.java | 2 +- .../spark/source/metrics/TaskSkippedDataManifests.java | 2 +- .../iceberg/spark/source/metrics/TaskTotalFileSize.java | 2 +- .../spark/source/metrics/TaskTotalPlanningDuration.java | 2 +- .../apache/iceberg/spark/source/metrics/TotalFileSize.java | 4 ++-- .../iceberg/spark/source/metrics/TotalPlanningDuration.java | 4 ++-- 12 files changed, 18 insertions(+), 18 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java index dccf9010de1c..a44bd495432a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java @@ -22,11 +22,11 @@ public class ResultDataFiles extends CustomSumMetric { - static final String name = "resultDataFiles"; + static final String NAME = "resultDataFiles"; @Override public String name() { - return name; + return NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java index 00421c5693ae..734050250a94 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -22,11 +22,11 @@ public class ScannedDataManifests extends CustomSumMetric { - static final String name = "scannedDataManifests"; + static final String NAME = "scannedDataManifests"; @Override public String name() { - return name; + return NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java index 7f01cf16f8f4..0b1fc2a60c5b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java @@ -22,11 +22,11 @@ public class SkippedDataFiles extends CustomSumMetric { - static final String name = "skippedDataFiles"; + static final String NAME = "skippedDataFiles"; @Override public String name() { - return name; + return NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java index bfef9988c3d8..794c13944901 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java @@ -22,11 +22,11 @@ public class SkippedDataManifests extends CustomSumMetric { - static final String name = "skippedDataManifests"; + static final String NAME = "skippedDataManifests"; @Override public String name() { - return name; + return NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java index c69fa59b6445..afcb971fe5ab 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java @@ -31,7 +31,7 @@ public TaskResultDataFiles(long value) { @Override public String name() { - return ResultDataFiles.name; + return ResultDataFiles.NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java index b9ed9e2d39c5..904b064db871 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java @@ -31,7 +31,7 @@ public TaskScannedDataManifests(long value) { @Override public String name() { - return ScannedDataManifests.name; + return ScannedDataManifests.NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java index 9c08964c6da2..c01698b52bed 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java @@ -31,7 +31,7 @@ public TaskSkippedDataFiles(long value) { @Override public String name() { - return SkippedDataFiles.name; + return SkippedDataFiles.NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java index 63d28bf76cd9..808c41d10749 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java @@ -31,7 +31,7 @@ public TaskSkippedDataManifests(long value) { @Override public String name() { - return SkippedDataManifests.name; + return SkippedDataManifests.NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java index df8fd66e1fda..d2b8faf6c1c4 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java @@ -32,7 +32,7 @@ public TaskTotalFileSize(long value) { @Override public String name() { - return TotalFileSize.name; + return TotalFileSize.NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java index b7b60b7e21d1..7241e6f8e841 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java @@ -32,7 +32,7 @@ public TaskTotalPlanningDuration(long value) { @Override public String name() { - return TotalPlanningDuration.name; + return TotalPlanningDuration.NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java index 84f918f91691..53875aa69472 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java @@ -23,11 +23,11 @@ public class TotalFileSize implements CustomMetric { - static final String name = "totalFileSize"; + static final String NAME = "totalFileSize"; @Override public String name() { - return name; + return NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java index a4249e2e7fce..7b684b767d8b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -23,11 +23,11 @@ public class TotalPlanningDuration implements CustomMetric { - static final String name = "totalPlanningDuration"; + static final String NAME = "totalPlanningDuration"; @Override public String name() { - return name; + return NAME; } @Override From ef0dc67bb730ed7fadc925f1db5ee63d423938fd Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Mon, 26 Jun 2023 16:01:00 -0700 Subject: [PATCH 08/14] Address review comments --- .../apache/iceberg}/metrics/InMemoryReadMetricReporter.java | 6 +----- .../java/org/apache/iceberg/spark/source/SparkScan.java | 1 - .../org/apache/iceberg/spark/source/SparkScanBuilder.java | 2 +- .../apache/iceberg/spark/source/SparkStagedScanBuilder.java | 2 +- 4 files changed, 3 insertions(+), 8 deletions(-) rename {spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source => core/src/main/java/org/apache/iceberg}/metrics/InMemoryReadMetricReporter.java (85%) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java b/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java similarity index 85% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java rename to core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java index 079e205dae37..9b4a73f53d82 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/InMemoryReadMetricReporter.java +++ b/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java @@ -16,11 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.spark.source.metrics; - -import org.apache.iceberg.metrics.MetricsReport; -import org.apache.iceberg.metrics.MetricsReporter; -import org.apache.iceberg.metrics.ScanReport; +package org.apache.iceberg.metrics; public class InMemoryReadMetricReporter implements MetricsReporter { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 44491c3e7d38..e6c4ee5a0a50 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -30,7 +30,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.metrics.ScanReport; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 5a0057d7d14a..13f3d59f9cd1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -43,6 +43,7 @@ import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.metrics.InMemoryReadMetricReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -52,7 +53,6 @@ import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.source.metrics.InMemoryReadMetricReporter; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 99e28724439b..ca4f75a7748e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -19,8 +19,8 @@ package org.apache.iceberg.spark.source; import org.apache.iceberg.Table; +import org.apache.iceberg.metrics.InMemoryReadMetricReporter; import org.apache.iceberg.spark.SparkReadConf; -import org.apache.iceberg.spark.source.metrics.InMemoryReadMetricReporter; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; From f9afa312e0061285ca09a4360e9bf61862e26bbd Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Wed, 5 Jul 2023 13:59:39 -0700 Subject: [PATCH 09/14] Address review comments --- .../iceberg/metrics/InMemoryReadMetricReporter.java | 12 +++++++----- .../iceberg/spark/source/SparkBatchQueryScan.java | 4 ++-- .../org/apache/iceberg/spark/source/SparkScan.java | 2 +- .../apache/iceberg/spark/source/SparkStagedScan.java | 10 ++-------- .../iceberg/spark/source/SparkStagedScanBuilder.java | 4 +--- .../iceberg/spark/source/metrics/TotalFileSize.java | 10 ++-------- .../spark/source/metrics/TotalPlanningDuration.java | 10 ++-------- 7 files changed, 17 insertions(+), 35 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java b/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java index 9b4a73f53d82..f34594d1023a 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java +++ b/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java @@ -18,18 +18,20 @@ */ package org.apache.iceberg.metrics; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + public class InMemoryReadMetricReporter implements MetricsReporter { - private ScanReport metricsReport; + private MetricsReport metricsReport; @Override public void report(MetricsReport report) { - if (report instanceof ScanReport) { - this.metricsReport = (ScanReport) report; - } + this.metricsReport = (ScanReport) report; } public ScanReport scanReport() { - return metricsReport; + Preconditions.checkArgument( + metricsReport instanceof ScanReport, "Metric report is not a scan report"); + return (ScanReport) metricsReport; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index 6f3622a7cfee..036c395f7115 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -76,8 +76,8 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan SparkReadConf readConf, Schema expectedSchema, List filters, - Supplier metricsReportSupplier) { - super(spark, table, scan, readConf, expectedSchema, filters, metricsReportSupplier); + Supplier scanReportSupplier) { + super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); this.snapshotId = readConf.snapshotId(); this.startSnapshotId = readConf.startSnapshotId(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index e6c4ee5a0a50..6f5ab25ae092 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -86,7 +86,6 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { Schema expectedSchema, List filters, Supplier metricsReportSupplier) { - this.metricsReportSupplier = metricsReportSupplier; Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); @@ -97,6 +96,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { this.expectedSchema = expectedSchema; this.filterExpressions = filters != null ? filters : Collections.emptyList(); this.branch = readConf.branch(); + this.metricsReportSupplier = metricsReportSupplier; } protected Table table() { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java index a0f5a954637c..0290bf7e84ce 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -20,12 +20,10 @@ import java.util.List; import java.util.Objects; -import java.util.function.Supplier; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkReadConf; @@ -41,12 +39,8 @@ class SparkStagedScan extends SparkScan { private List> taskGroups = null; // lazy cache of tasks - SparkStagedScan( - SparkSession spark, - Table table, - SparkReadConf readConf, - Supplier scanReportSupplier) { - super(spark, table, readConf, table.schema(), ImmutableList.of(), scanReportSupplier); + SparkStagedScan(SparkSession spark, Table table, SparkReadConf readConf) { + super(spark, table, readConf, table.schema(), ImmutableList.of(), null); this.taskSetId = readConf.scanTaskSetId(); this.splitSize = readConf.splitSize(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index ca4f75a7748e..37bbea42e5b1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import org.apache.iceberg.Table; -import org.apache.iceberg.metrics.InMemoryReadMetricReporter; import org.apache.iceberg.spark.SparkReadConf; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; @@ -40,7 +39,6 @@ class SparkStagedScanBuilder implements ScanBuilder { @Override public Scan build() { - return new SparkStagedScan( - spark, table, readConf, (new InMemoryReadMetricReporter())::scanReport); + return new SparkStagedScan(spark, table, readConf); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java index 53875aa69472..1b82325605df 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.Arrays; -import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class TotalFileSize implements CustomMetric { +public class TotalFileSize extends CustomSumMetric { static final String NAME = "totalFileSize"; @@ -34,9 +33,4 @@ public String name() { public String description() { return "total file size"; } - - @Override - public String aggregateTaskMetrics(long[] taskMetrics) { - return String.valueOf(Arrays.stream(taskMetrics).sum()); - } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java index 7b684b767d8b..976bea350364 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.spark.source.metrics; -import java.util.Arrays; -import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class TotalPlanningDuration implements CustomMetric { +public class TotalPlanningDuration extends CustomSumMetric { static final String NAME = "totalPlanningDuration"; @@ -34,9 +33,4 @@ public String name() { public String description() { return "total planning duration"; } - - @Override - public String aggregateTaskMetrics(long[] taskMetrics) { - return String.valueOf(Arrays.stream(taskMetrics).sum()); - } } From 01fef743560b60337f38d69dfda6fa30c4b28030 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Thu, 13 Jul 2023 10:34:18 -0700 Subject: [PATCH 10/14] Address review commenst --- ...rter.java => InMemoryMetricsReporter.java} | 7 ++-- .../source/SparkPartitioningAwareScan.java | 4 +-- .../iceberg/spark/source/SparkScan.java | 8 ++--- .../spark/source/SparkScanBuilder.java | 6 ++-- .../source/metrics/ScannedDataManifests.java | 9 +++++ .../source/metrics/TaskResultDataFiles.java | 2 +- .../metrics/TaskScannedDataManifests.java | 2 +- .../source/metrics/TaskSkippedDataFiles.java | 2 +- .../metrics/TaskSkippedDataManifests.java | 2 +- .../source/metrics/TaskTotalFileSize.java | 2 +- .../metrics/TaskTotalPlanningDuration.java | 6 ++-- .../spark/source/TestSparkReadMetrics.java | 36 ++++++++++++++++--- 12 files changed, 61 insertions(+), 25 deletions(-) rename core/src/main/java/org/apache/iceberg/metrics/{InMemoryReadMetricReporter.java => InMemoryMetricsReporter.java} (84%) diff --git a/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java b/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java similarity index 84% rename from core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java rename to core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java index f34594d1023a..b66f7faef8f3 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/InMemoryReadMetricReporter.java +++ b/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java @@ -20,18 +20,19 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -public class InMemoryReadMetricReporter implements MetricsReporter { +public class InMemoryMetricsReporter implements MetricsReporter { private MetricsReport metricsReport; @Override public void report(MetricsReport report) { - this.metricsReport = (ScanReport) report; + this.metricsReport = report; } public ScanReport scanReport() { Preconditions.checkArgument( - metricsReport instanceof ScanReport, "Metric report is not a scan report"); + metricsReport == null || metricsReport instanceof ScanReport, + "Metric report is not a scan report"); return (ScanReport) metricsReport; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java index eae901ff1caa..0bf8c9016738 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java @@ -77,9 +77,9 @@ abstract class SparkPartitioningAwareScan extends S SparkReadConf readConf, Schema expectedSchema, List filters, - Supplier metricsReportSupplier) { + Supplier scanReportSupplier) { - super(spark, table, readConf, expectedSchema, filters, metricsReportSupplier); + super(spark, table, readConf, expectedSchema, filters, scanReportSupplier); this.scan = scan; this.preserveDataGrouping = readConf.preserveDataGrouping(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 6f5ab25ae092..1e4ef82011d3 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -74,7 +74,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { private final Schema expectedSchema; private final List filterExpressions; private final String branch; - private final Supplier metricsReportSupplier; + private final Supplier scanReportSupplier; // lazy variables private StructType readSchema; @@ -85,7 +85,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { SparkReadConf readConf, Schema expectedSchema, List filters, - Supplier metricsReportSupplier) { + Supplier scanReportSupplier) { Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); @@ -96,7 +96,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { this.expectedSchema = expectedSchema; this.filterExpressions = filters != null ? filters : Collections.emptyList(); this.branch = readConf.branch(); - this.metricsReportSupplier = metricsReportSupplier; + this.scanReportSupplier = scanReportSupplier; } protected Table table() { @@ -191,7 +191,7 @@ public String description() { @Override public CustomTaskMetric[] reportDriverMetrics() { - ScanReport scanReport = metricsReportSupplier != null ? metricsReportSupplier.get() : null; + ScanReport scanReport = scanReportSupplier != null ? scanReportSupplier.get() : null; if (scanReport == null) { return new CustomTaskMetric[0]; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 13f3d59f9cd1..546f1076b0a9 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -43,7 +43,7 @@ import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.metrics.InMemoryReadMetricReporter; +import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -92,12 +92,12 @@ public class SparkScanBuilder private final CaseInsensitiveStringMap options; private final SparkReadConf readConf; private final List metaColumns = Lists.newArrayList(); + private final InMemoryMetricsReporter metricsReporter; private Schema schema = null; private boolean caseSensitive; private List filterExpressions = null; private Filter[] pushedFilters = NO_FILTERS; - private final InMemoryReadMetricReporter metricsReporter; SparkScanBuilder( SparkSession spark, @@ -111,7 +111,7 @@ public class SparkScanBuilder this.options = options; this.readConf = new SparkReadConf(spark, table, branch, options); this.caseSensitive = readConf.caseSensitive(); - this.metricsReporter = new InMemoryReadMetricReporter(); + this.metricsReporter = new InMemoryMetricsReporter(); } SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java index 734050250a94..5dea041123b6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -33,4 +33,13 @@ public String name() { public String description() { return "num scanned data manifests"; } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + long sum = 0L; + for (long taskMetric : taskMetrics) { + sum += taskMetric; + } + return String.valueOf(sum); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java index afcb971fe5ab..7b12e55054b6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java @@ -25,7 +25,7 @@ public class TaskResultDataFiles implements CustomTaskMetric { private final long value; - public TaskResultDataFiles(long value) { + private TaskResultDataFiles(long value) { this.value = value; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java index 904b064db871..cb28b59a61ec 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java @@ -25,7 +25,7 @@ public class TaskScannedDataManifests implements CustomTaskMetric { private final long value; - public TaskScannedDataManifests(long value) { + private TaskScannedDataManifests(long value) { this.value = value; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java index c01698b52bed..69d877229fb1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java @@ -25,7 +25,7 @@ public class TaskSkippedDataFiles implements CustomTaskMetric { private final long value; - public TaskSkippedDataFiles(long value) { + private TaskSkippedDataFiles(long value) { this.value = value; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java index 808c41d10749..e5c0db046932 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java @@ -25,7 +25,7 @@ public class TaskSkippedDataManifests implements CustomTaskMetric { private final long value; - public TaskSkippedDataManifests(long value) { + private TaskSkippedDataManifests(long value) { this.value = value; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java index d2b8faf6c1c4..eda790d91921 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java @@ -26,7 +26,7 @@ public class TaskTotalFileSize implements CustomTaskMetric { private final long value; - public TaskTotalFileSize(long value) { + private TaskTotalFileSize(long value) { this.value = value; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java index 7241e6f8e841..17b76bc7aaf2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java @@ -26,7 +26,7 @@ public class TaskTotalPlanningDuration implements CustomTaskMetric { private final long value; - public TaskTotalPlanningDuration(long value) { + private TaskTotalPlanningDuration(long value) { this.value = value; } @@ -40,9 +40,9 @@ public long value() { return value; } - public static TaskTotalFileSize from(ScanReport scanReport) { + public static TaskTotalPlanningDuration from(ScanReport scanReport) { TimerResult timerResult = scanReport.scanMetrics().totalPlanningDuration(); long value = timerResult != null ? timerResult.count() : -1; - return new TaskTotalFileSize(value); + return new TaskTotalPlanningDuration(value); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index 782eb9697e9a..24f648fbad88 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -28,9 +28,9 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.metric.SQLMetric; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; -import org.junit.jupiter.api.Assertions; import scala.collection.JavaConverters; public class TestSparkReadMetrics extends SparkTestBaseWithCatalog { @@ -41,18 +41,44 @@ public void removeTables() { } @Test - public void testReadMetrics() throws NoSuchTableException { - sql("CREATE TABLE %s (id BIGINT) USING iceberg", tableName); + public void testReadMetricsForV1Table() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='2')", + tableName); spark.range(10000).coalesce(1).writeTo(tableName).append(); + spark.range(10001, 20000).coalesce(1).writeTo(tableName).append(); - Dataset df = spark.sql(String.format("select * from %s", tableName)); + Dataset df = spark.sql(String.format("select * from %s where id < 10000", tableName)); df.collect(); List sparkPlans = seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); Map metricsMap = JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); - Assertions.assertEquals(1, metricsMap.get("scannedDataManifests").value()); + Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + } + + @Test + public void testReadMetricsForV2Table() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='2')", + tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + spark.range(10001, 20000).coalesce(1).writeTo(tableName).append(); + + Dataset df = spark.sql(String.format("select * from %s where id < 10000", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); } } From 57ce79f1a2d17fc3eb1839b44109df4c3db2bfe0 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Thu, 13 Jul 2023 15:18:12 -0700 Subject: [PATCH 11/14] Address review comments --- .../org/apache/iceberg/spark/source/SparkScan.java | 8 ++++---- .../spark/source/metrics/ScannedDataManifests.java | 2 +- .../spark/source/metrics/SkippedDataFiles.java | 2 +- .../spark/source/metrics/SkippedDataManifests.java | 2 +- ...esultDataFiles.java => TaskScannedDataFiles.java} | 12 ++++++------ .../source/metrics/TaskScannedDataManifests.java | 2 +- .../spark/source/metrics/TaskSkippedDataFiles.java | 2 +- .../source/metrics/TaskSkippedDataManifests.java | 2 +- .../spark/source/metrics/TaskTotalFileSize.java | 2 +- .../{ResultDataFiles.java => scannedDataFiles.java} | 6 +++--- 10 files changed, 20 insertions(+), 20 deletions(-) rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{TaskResultDataFiles.java => TaskScannedDataFiles.java} (80%) rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{ResultDataFiles.java => scannedDataFiles.java} (87%) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 1e4ef82011d3..f648f7e6e7ae 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -36,11 +36,10 @@ import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.source.metrics.NumDeletes; import org.apache.iceberg.spark.source.metrics.NumSplits; -import org.apache.iceberg.spark.source.metrics.ResultDataFiles; import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; -import org.apache.iceberg.spark.source.metrics.TaskResultDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskScannedDataFiles; import org.apache.iceberg.spark.source.metrics.TaskScannedDataManifests; import org.apache.iceberg.spark.source.metrics.TaskSkippedDataFiles; import org.apache.iceberg.spark.source.metrics.TaskSkippedDataManifests; @@ -48,6 +47,7 @@ import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; import org.apache.iceberg.spark.source.metrics.TotalFileSize; import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; +import org.apache.iceberg.spark.source.metrics.scannedDataFiles; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; @@ -200,7 +200,7 @@ public CustomTaskMetric[] reportDriverMetrics() { driverMetrics.add(TaskTotalFileSize.from(scanReport)); driverMetrics.add(TaskTotalPlanningDuration.from(scanReport)); driverMetrics.add(TaskSkippedDataFiles.from(scanReport)); - driverMetrics.add(TaskResultDataFiles.from(scanReport)); + driverMetrics.add(TaskScannedDataFiles.from(scanReport)); driverMetrics.add(TaskSkippedDataManifests.from(scanReport)); driverMetrics.add(TaskScannedDataManifests.from(scanReport)); @@ -216,7 +216,7 @@ public CustomMetric[] supportedCustomMetrics() { new TotalPlanningDuration(), new ScannedDataManifests(), new SkippedDataManifests(), - new ResultDataFiles(), + new scannedDataFiles(), new SkippedDataFiles() }; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java index 5dea041123b6..1e0061297864 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -31,7 +31,7 @@ public String name() { @Override public String description() { - return "num scanned data manifests"; + return "number of scanned data manifests"; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java index 0b1fc2a60c5b..7fd17425313d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java @@ -31,6 +31,6 @@ public String name() { @Override public String description() { - return "num skipped data files"; + return "number of skipped data files"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java index 794c13944901..b0eaeb5d87f2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java @@ -31,6 +31,6 @@ public String name() { @Override public String description() { - return "num skipped data manifests"; + return "number of skipped data manifests"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java similarity index 80% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java index 7b12e55054b6..d18cc47afc6c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java @@ -22,16 +22,16 @@ import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; -public class TaskResultDataFiles implements CustomTaskMetric { +public class TaskScannedDataFiles implements CustomTaskMetric { private final long value; - private TaskResultDataFiles(long value) { + private TaskScannedDataFiles(long value) { this.value = value; } @Override public String name() { - return ResultDataFiles.NAME; + return scannedDataFiles.NAME; } @Override @@ -39,9 +39,9 @@ public long value() { return value; } - public static TaskResultDataFiles from(ScanReport scanReport) { + public static TaskScannedDataFiles from(ScanReport scanReport) { CounterResult counter = scanReport.scanMetrics().resultDataFiles(); - long value = counter != null ? counter.value() : -1; - return new TaskResultDataFiles(value); + long value = counter != null ? counter.value() : 0L; + return new TaskScannedDataFiles(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java index cb28b59a61ec..09dd0339910c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java @@ -41,7 +41,7 @@ public long value() { public static TaskScannedDataManifests from(ScanReport scanReport) { CounterResult counter = scanReport.scanMetrics().scannedDataManifests(); - long value = counter != null ? counter.value() : -1; + long value = counter != null ? counter.value() : 0L; return new TaskScannedDataManifests(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java index 69d877229fb1..5165f9a3116c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java @@ -41,7 +41,7 @@ public long value() { public static TaskSkippedDataFiles from(ScanReport scanReport) { CounterResult counter = scanReport.scanMetrics().skippedDataFiles(); - long value = counter != null ? counter.value() : -1; + long value = counter != null ? counter.value() : 0L; return new TaskSkippedDataFiles(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java index e5c0db046932..86fef8c4118b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java @@ -41,7 +41,7 @@ public long value() { public static TaskSkippedDataManifests from(ScanReport scanReport) { CounterResult counter = scanReport.scanMetrics().skippedDataManifests(); - long value = counter != null ? counter.value() : -1; + long value = counter != null ? counter.value() : 0L; return new TaskSkippedDataManifests(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java index eda790d91921..c300d835e777 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java @@ -42,7 +42,7 @@ public long value() { public static TaskTotalFileSize from(ScanReport scanReport) { CounterResult counter = scanReport.scanMetrics().totalFileSizeInBytes(); - long value = counter != null ? counter.value() : -1; + long value = counter != null ? counter.value() : 0L; return new TaskTotalFileSize(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/scannedDataFiles.java similarity index 87% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/scannedDataFiles.java index a44bd495432a..296ec52f86f8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/scannedDataFiles.java @@ -20,9 +20,9 @@ import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class ResultDataFiles extends CustomSumMetric { +public class scannedDataFiles extends CustomSumMetric { - static final String NAME = "resultDataFiles"; + static final String NAME = "scannedDataFiles"; @Override public String name() { @@ -31,6 +31,6 @@ public String name() { @Override public String description() { - return "result data files"; + return "number of scanned data files"; } } From 760b67fac4bcc36dcda3f7907a749bd952a64d12 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Fri, 14 Jul 2023 06:11:33 -0700 Subject: [PATCH 12/14] Add review comments --- .../iceberg/spark/source/SparkCopyOnWriteScan.java | 1 - .../iceberg/spark/source/SparkPartitioningAwareScan.java | 1 - .../java/org/apache/iceberg/spark/source/SparkScan.java | 5 +++-- .../{scannedDataFiles.java => ScannedDataFiles.java} | 2 +- .../spark/source/metrics/ScannedDataManifests.java | 9 --------- .../spark/source/metrics/TaskScannedDataFiles.java | 2 +- .../spark/source/metrics/TaskTotalPlanningDuration.java | 2 +- .../iceberg/spark/source/metrics/TotalFileSize.java | 2 +- .../spark/source/metrics/TotalPlanningDuration.java | 2 +- .../iceberg/spark/source/TestSparkReadMetrics.java | 4 ++-- 10 files changed, 10 insertions(+), 20 deletions(-) rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{scannedDataFiles.java => ScannedDataFiles.java} (95%) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index e32b68fb054f..16eb9c51df5c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -73,7 +73,6 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan Schema expectedSchema, List filters, Supplier scanReportSupplier) { - super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); this.snapshot = snapshot; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java index 0bf8c9016738..6538268697e2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java @@ -78,7 +78,6 @@ abstract class SparkPartitioningAwareScan extends S Schema expectedSchema, List filters, Supplier scanReportSupplier) { - super(spark, table, readConf, expectedSchema, filters, scanReportSupplier); this.scan = scan; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index f648f7e6e7ae..65c5e04f31a1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -36,6 +36,7 @@ import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.source.metrics.NumDeletes; import org.apache.iceberg.spark.source.metrics.NumSplits; +import org.apache.iceberg.spark.source.metrics.ScannedDataFiles; import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; @@ -47,7 +48,6 @@ import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; import org.apache.iceberg.spark.source.metrics.TotalFileSize; import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; -import org.apache.iceberg.spark.source.metrics.scannedDataFiles; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; @@ -192,6 +192,7 @@ public String description() { @Override public CustomTaskMetric[] reportDriverMetrics() { ScanReport scanReport = scanReportSupplier != null ? scanReportSupplier.get() : null; + if (scanReport == null) { return new CustomTaskMetric[0]; } @@ -216,7 +217,7 @@ public CustomMetric[] supportedCustomMetrics() { new TotalPlanningDuration(), new ScannedDataManifests(), new SkippedDataManifests(), - new scannedDataFiles(), + new ScannedDataFiles(), new SkippedDataFiles() }; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/scannedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java similarity index 95% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/scannedDataFiles.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java index 296ec52f86f8..f453872fdc29 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/scannedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java @@ -20,7 +20,7 @@ import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class scannedDataFiles extends CustomSumMetric { +public class ScannedDataFiles extends CustomSumMetric { static final String NAME = "scannedDataFiles"; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java index 1e0061297864..a167904280e6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -33,13 +33,4 @@ public String name() { public String description() { return "number of scanned data manifests"; } - - @Override - public String aggregateTaskMetrics(long[] taskMetrics) { - long sum = 0L; - for (long taskMetric : taskMetrics) { - sum += taskMetric; - } - return String.valueOf(sum); - } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java index d18cc47afc6c..d9a527da08f6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java @@ -31,7 +31,7 @@ private TaskScannedDataFiles(long value) { @Override public String name() { - return scannedDataFiles.NAME; + return ScannedDataFiles.NAME; } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java index 17b76bc7aaf2..32ac6fde8bf3 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java @@ -42,7 +42,7 @@ public long value() { public static TaskTotalPlanningDuration from(ScanReport scanReport) { TimerResult timerResult = scanReport.scanMetrics().totalPlanningDuration(); - long value = timerResult != null ? timerResult.count() : -1; + long value = timerResult != null ? timerResult.totalDuration().toMillis() : -1; return new TaskTotalPlanningDuration(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java index 1b82325605df..994626e54f10 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java @@ -31,6 +31,6 @@ public String name() { @Override public String description() { - return "total file size"; + return "total file size (bytes)"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java index 976bea350364..8b66eeac4046 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -31,6 +31,6 @@ public String name() { @Override public String description() { - return "total planning duration"; + return "total planning duration (ms)"; } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index 24f648fbad88..367e4da043fb 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -58,7 +58,7 @@ public void testReadMetricsForV1Table() throws NoSuchTableException { JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); } @Test @@ -79,6 +79,6 @@ public void testReadMetricsForV2Table() throws NoSuchTableException { JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); } } From 831a170825f7091141b0b45e730fc46af78f252c Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Wed, 19 Jul 2023 13:18:51 -0700 Subject: [PATCH 13/14] Address review comments --- .../org/apache/iceberg/metrics/InMemoryMetricsReporter.java | 2 +- .../apache/iceberg/spark/source/TestSparkReadMetrics.java | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java b/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java index b66f7faef8f3..79b446c0ddbf 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java +++ b/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java @@ -32,7 +32,7 @@ public void report(MetricsReport report) { public ScanReport scanReport() { Preconditions.checkArgument( metricsReport == null || metricsReport instanceof ScanReport, - "Metric report is not a scan report"); + "Metrics report is not a scan report"); return (ScanReport) metricsReport; } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index 367e4da043fb..7b943372d167 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -59,6 +59,9 @@ public void testReadMetricsForV1Table() throws NoSuchTableException { Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("totalFileSize").value()).isNotEqualTo(0); + Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); } @Test @@ -80,5 +83,8 @@ public void testReadMetricsForV2Table() throws NoSuchTableException { Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("totalFileSize").value()).isNotEqualTo(0); + Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); } } From 58ec63e180d68d842f3e5d186755a8f4a87f4d40 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Thu, 20 Jul 2023 14:26:14 -0700 Subject: [PATCH 14/14] Fix test- Thanks Anton --- core/src/main/java/org/apache/iceberg/BaseScan.java | 2 +- .../java/org/apache/iceberg/spark/source/SparkScanBuilder.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 9db72227ac3f..953ad754aa1b 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -254,6 +254,6 @@ private static Schema lazyColumnProjection(TableScanContext context, Schema sche @Override public ThisT metricsReporter(MetricsReporter reporter) { - return newRefinedScan(table(), schema(), context().reportWith(reporter)); + return newRefinedScan(table, schema, context.reportWith(reporter)); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 546f1076b0a9..d19368cec819 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -453,6 +453,7 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, } scan = configureSplitPlanning(scan); + return new SparkBatchQueryScan( spark, table,