Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Display Spark read metrics on Spark SQL UI #7447

Merged
merged 14 commits into from Jul 20, 2023
2 changes: 1 addition & 1 deletion core/src/main/java/org/apache/iceberg/BaseScan.java
Expand Up @@ -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));
}
}
@@ -0,0 +1,38 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.metrics;

import org.apache.iceberg.relocated.com.google.common.base.Preconditions;

public class InMemoryMetricsReporter implements MetricsReporter {

private MetricsReport metricsReport;

@Override
public void report(MetricsReport report) {
this.metricsReport = report;
}

public ScanReport scanReport() {
Preconditions.checkArgument(
metricsReport == null || metricsReport instanceof ScanReport,
"Metrics report is not a scan report");
return (ScanReport) metricsReport;
}
}
Expand Up @@ -22,6 +22,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.iceberg.PartitionField;
import org.apache.iceberg.PartitionScanTask;
Expand All @@ -39,6 +40,7 @@
import org.apache.iceberg.expressions.ExpressionUtil;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.expressions.Projections;
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;
Expand Down Expand Up @@ -73,9 +75,9 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan<PartitionScanTask>
Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
SparkReadConf readConf,
Schema expectedSchema,
List<Expression> filters) {

super(spark, table, scan, readConf, expectedSchema, filters);
List<Expression> filters,
Supplier<ScanReport> scanReportSupplier) {
super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier);

this.snapshotId = readConf.snapshotId();
this.startSnapshotId = readConf.startSnapshotId();
Expand Down
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -57,8 +59,9 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan<FileScanTask>
Table table,
SparkReadConf readConf,
Schema expectedSchema,
List<Expression> filters) {
this(spark, table, null, null, readConf, expectedSchema, filters);
List<Expression> filters,
Supplier<ScanReport> scanReportSupplier) {
this(spark, table, null, null, readConf, expectedSchema, filters, scanReportSupplier);
}

SparkCopyOnWriteScan(
Expand All @@ -68,9 +71,9 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan<FileScanTask>
Snapshot snapshot,
SparkReadConf readConf,
Schema expectedSchema,
List<Expression> filters) {

super(spark, table, scan, readConf, expectedSchema, filters);
aokolnychyi marked this conversation as resolved.
Show resolved Hide resolved
List<Expression> filters,
Supplier<ScanReport> scanReportSupplier) {
super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier);

this.snapshot = snapshot;

Expand Down
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -74,9 +76,9 @@ abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends S
Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
SparkReadConf readConf,
Schema expectedSchema,
List<Expression> filters) {

super(spark, table, readConf, expectedSchema, filters);
List<Expression> filters,
Supplier<ScanReport> scanReportSupplier) {
super(spark, table, readConf, expectedSchema, filters, scanReportSupplier);

this.scan = scan;
this.preserveDataGrouping = readConf.preserveDataGrouping();
Expand Down
Expand Up @@ -21,24 +21,40 @@
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;
import org.apache.iceberg.Snapshot;
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.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.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;
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;
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;
Expand All @@ -58,6 +74,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics {
private final Schema expectedSchema;
private final List<Expression> filterExpressions;
private final String branch;
private final Supplier<ScanReport> scanReportSupplier;

// lazy variables
private StructType readSchema;
Expand All @@ -67,7 +84,8 @@ abstract class SparkScan implements Scan, SupportsReportStatistics {
Table table,
SparkReadConf readConf,
Schema expectedSchema,
List<Expression> filters) {
List<Expression> filters,
Supplier<ScanReport> scanReportSupplier) {
Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch());
SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema);

Expand All @@ -78,6 +96,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics {
this.expectedSchema = expectedSchema;
this.filterExpressions = filters != null ? filters : Collections.emptyList();
this.branch = readConf.branch();
this.scanReportSupplier = scanReportSupplier;
}

protected Table table() {
Expand Down Expand Up @@ -170,8 +189,36 @@ public String description() {
table(), branch(), Spark3Util.describe(filterExpressions), groupingKeyFieldNamesAsString);
}

@Override
public CustomTaskMetric[] reportDriverMetrics() {
ScanReport scanReport = scanReportSupplier != null ? scanReportSupplier.get() : null;

if (scanReport == null) {
aokolnychyi marked this conversation as resolved.
Show resolved Hide resolved
aokolnychyi marked this conversation as resolved.
Show resolved Hide resolved
return new CustomTaskMetric[0];
}

List<CustomTaskMetric> driverMetrics = Lists.newArrayList();
driverMetrics.add(TaskTotalFileSize.from(scanReport));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a reason why we don't include all metrics from ScanMetricsResult?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had added all metrics from ScanMetricsResult at the point in time when the change was done,
Should we add the remaining now or take in a different PR(since this cange is already reviewed almost)?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is a list of the metrics:

TimerResult totalPlanningDuration(); // DONE
CounterResult resultDataFiles(); // DONE
CounterResult resultDeleteFiles(); // MISSING
CounterResult totalDataManifests(); // MISSING
CounterResult totalDeleteManifests(); // MISSING
CounterResult scannedDataManifests(); // DONE
CounterResult skippedDataManifests(); // DONE
CounterResult totalFileSizeInBytes(); // DONE
CounterResult totalDeleteFileSizeInBytes(); // MISSING
CounterResult skippedDataFiles(); // DONE
CounterResult skippedDeleteFiles(); // MISSING
CounterResult scannedDeleteManifests(); // MISSING
CounterResult skippedDeleteManifests(); // MISSING
CounterResult indexedDeleteFiles(); // MISSING
CounterResult equalityDeleteFiles(); // MISSING
CounterResult positionalDeleteFiles(); // MISSING

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can be done in a follow-up.

driverMetrics.add(TaskTotalPlanningDuration.from(scanReport));
driverMetrics.add(TaskSkippedDataFiles.from(scanReport));
driverMetrics.add(TaskScannedDataFiles.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 ScannedDataFiles(),
new SkippedDataFiles()
};
}
}
Expand Up @@ -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.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;
Expand Down Expand Up @@ -91,6 +92,7 @@ public class SparkScanBuilder
private final CaseInsensitiveStringMap options;
private final SparkReadConf readConf;
private final List<String> metaColumns = Lists.newArrayList();
private final InMemoryMetricsReporter metricsReporter;

private Schema schema = null;
private boolean caseSensitive;
Expand All @@ -109,6 +111,7 @@ public class SparkScanBuilder
this.options = options;
this.readConf = new SparkReadConf(spark, table, branch, options);
this.caseSensitive = readConf.caseSensitive();
this.metricsReporter = new InMemoryMetricsReporter();
}

SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) {
Expand Down Expand Up @@ -430,7 +433,8 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch,
.newBatchScan()
.caseSensitive(caseSensitive)
.filter(filterExpression())
.project(expectedSchema);
.project(expectedSchema)
.metricsReporter(metricsReporter);

if (snapshotId != null) {
scan = scan.useSnapshot(snapshotId);
Expand All @@ -450,7 +454,14 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch,

scan = configureSplitPlanning(scan);

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we keep this empty line?

return new SparkBatchQueryScan(spark, table, scan, readConf, expectedSchema, filterExpressions);
return new SparkBatchQueryScan(
spark,
table,
scan,
readConf,
expectedSchema,
filterExpressions,
metricsReporter::scanReport);
}

private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId) {
Expand All @@ -470,7 +481,14 @@ 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,
metricsReporter::scanReport);
}

public Scan buildChangelogScan() {
Expand Down Expand Up @@ -573,7 +591,13 @@ public Scan buildMergeOnReadScan() {

if (snapshot == null) {
return new SparkBatchQueryScan(
spark, table, null, readConf, schemaWithMetadataColumns(), filterExpressions);
spark,
table,
null,
readConf,
schemaWithMetadataColumns(),
filterExpressions,
metricsReporter::scanReport);
}

// remember the current snapshot ID for commit validation
Expand All @@ -597,15 +621,26 @@ public Scan buildMergeOnReadScan() {
scan = configureSplitPlanning(scan);

return new SparkBatchQueryScan(
spark, table, scan, adjustedReadConf, expectedSchema, filterExpressions);
spark,
table,
scan,
adjustedReadConf,
expectedSchema,
filterExpressions,
metricsReporter::scanReport);
}

public Scan buildCopyOnWriteScan() {
Snapshot snapshot = SnapshotUtil.latestSnapshot(table, readConf.branch());

if (snapshot == null) {
return new SparkCopyOnWriteScan(
spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
spark,
table,
readConf,
schemaWithMetadataColumns(),
filterExpressions,
metricsReporter::scanReport);
}

Schema expectedSchema = schemaWithMetadataColumns();
Expand All @@ -622,7 +657,14 @@ public Scan buildCopyOnWriteScan() {
scan = configureSplitPlanning(scan);

return new SparkCopyOnWriteScan(
spark, table, scan, snapshot, readConf, expectedSchema, filterExpressions);
spark,
table,
scan,
snapshot,
readConf,
expectedSchema,
filterExpressions,
metricsReporter::scanReport);
}

private <T extends org.apache.iceberg.Scan<T, ?, ?>> T configureSplitPlanning(T scan) {
Expand Down
Expand Up @@ -40,7 +40,7 @@ class SparkStagedScan extends SparkScan {
private List<ScanTaskGroup<ScanTask>> taskGroups = null; // lazy cache of tasks

SparkStagedScan(SparkSession spark, Table table, SparkReadConf readConf) {
super(spark, table, readConf, table.schema(), ImmutableList.of());
super(spark, table, readConf, table.schema(), ImmutableList.of(), null);

this.taskSetId = readConf.scanTaskSetId();
this.splitSize = readConf.splitSize();
Expand Down