/
SparkScan.java
224 lines (196 loc) · 8.04 KB
/
SparkScan.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
/*
* 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 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;
import org.apache.spark.sql.connector.read.SupportsReportStatistics;
import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
import org.apache.spark.sql.types.StructType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
abstract class SparkScan implements Scan, SupportsReportStatistics {
private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class);
private final JavaSparkContext sparkContext;
private final Table table;
private final SparkReadConf readConf;
private final boolean caseSensitive;
private final Schema expectedSchema;
private final List<Expression> filterExpressions;
private final String branch;
private final Supplier<ScanReport> scanReportSupplier;
// lazy variables
private StructType readSchema;
SparkScan(
SparkSession spark,
Table table,
SparkReadConf readConf,
Schema expectedSchema,
List<Expression> filters,
Supplier<ScanReport> scanReportSupplier) {
Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch());
SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema);
this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
this.table = table;
this.readConf = readConf;
this.caseSensitive = readConf.caseSensitive();
this.expectedSchema = expectedSchema;
this.filterExpressions = filters != null ? filters : Collections.emptyList();
this.branch = readConf.branch();
this.scanReportSupplier = scanReportSupplier;
}
protected Table table() {
return table;
}
protected String branch() {
return branch;
}
protected boolean caseSensitive() {
return caseSensitive;
}
protected Schema expectedSchema() {
return expectedSchema;
}
protected List<Expression> filterExpressions() {
return filterExpressions;
}
protected Types.StructType groupingKeyType() {
return Types.StructType.of();
}
protected abstract List<? extends ScanTaskGroup<?>> taskGroups();
@Override
public Batch toBatch() {
return new SparkBatch(
sparkContext, table, readConf, groupingKeyType(), taskGroups(), expectedSchema, hashCode());
}
@Override
public MicroBatchStream toMicroBatchStream(String checkpointLocation) {
return new SparkMicroBatchStream(
sparkContext, table, readConf, expectedSchema, checkpointLocation);
}
@Override
public StructType readSchema() {
if (readSchema == null) {
this.readSchema = SparkSchemaUtil.convert(expectedSchema);
}
return readSchema;
}
@Override
public Statistics estimateStatistics() {
return estimateStatistics(SnapshotUtil.latestSnapshot(table, branch));
}
protected Statistics estimateStatistics(Snapshot snapshot) {
// its a fresh table, no data
if (snapshot == null) {
return new Stats(0L, 0L);
}
// estimate stats using snapshot summary only for partitioned tables
// (metadata tables are unpartitioned)
if (!table.spec().isUnpartitioned() && filterExpressions.isEmpty()) {
LOG.debug(
"Using snapshot {} metadata to estimate statistics for table {}",
snapshot.snapshotId(),
table.name());
long totalRecords = totalRecords(snapshot);
return new Stats(SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords);
}
long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum();
long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount);
return new Stats(sizeInBytes, rowsCount);
}
private long totalRecords(Snapshot snapshot) {
Map<String, String> summary = snapshot.summary();
return PropertyUtil.propertyAsLong(summary, SnapshotSummary.TOTAL_RECORDS_PROP, Long.MAX_VALUE);
}
@Override
public String description() {
String groupingKeyFieldNamesAsString =
groupingKeyType().fields().stream()
.map(Types.NestedField::name)
.collect(Collectors.joining(", "));
return String.format(
"%s (branch=%s) [filters=%s, groupedBy=%s]",
table(), branch(), Spark3Util.describe(filterExpressions), groupingKeyFieldNamesAsString);
}
@Override
public CustomTaskMetric[] reportDriverMetrics() {
ScanReport scanReport = scanReportSupplier != null ? scanReportSupplier.get() : null;
if (scanReport == null) {
return new CustomTaskMetric[0];
}
List<CustomTaskMetric> driverMetrics = Lists.newArrayList();
driverMetrics.add(TaskTotalFileSize.from(scanReport));
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(),
new TotalFileSize(),
new TotalPlanningDuration(),
new ScannedDataManifests(),
new SkippedDataManifests(),
new ScannedDataFiles(),
new SkippedDataFiles()
};
}
}