forked from apache/iceberg
/
SparkBatchQueryScan.java
261 lines (227 loc) · 9.24 KB
/
SparkBatchQueryScan.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
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.spark.source;
import java.util.List;
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;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Scan;
import org.apache.iceberg.ScanTask;
import org.apache.iceberg.ScanTaskGroup;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.Table;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.expressions.Binder;
import org.apache.iceberg.expressions.Evaluator;
import org.apache.iceberg.expressions.Expression;
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;
import org.apache.iceberg.spark.Spark3Util;
import org.apache.iceberg.spark.SparkFilters;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.iceberg.spark.SparkSchemaUtil;
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.read.Statistics;
import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering;
import org.apache.spark.sql.sources.Filter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class SparkBatchQueryScan extends SparkPartitioningAwareScan<PartitionScanTask>
implements SupportsRuntimeFiltering {
private static final Logger LOG = LoggerFactory.getLogger(SparkBatchQueryScan.class);
private final Long snapshotId;
private final Long startSnapshotId;
private final Long endSnapshotId;
private final Long asOfTimestamp;
private final String tag;
private final List<Expression> runtimeFilterExpressions;
SparkBatchQueryScan(
SparkSession spark,
Table table,
Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
SparkReadConf readConf,
Schema expectedSchema,
List<Expression> filters,
Supplier<ScanReport> scanReportSupplier) {
super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier);
this.snapshotId = readConf.snapshotId();
this.startSnapshotId = readConf.startSnapshotId();
this.endSnapshotId = readConf.endSnapshotId();
this.asOfTimestamp = readConf.asOfTimestamp();
this.tag = readConf.tag();
this.runtimeFilterExpressions = Lists.newArrayList();
}
Long snapshotId() {
return snapshotId;
}
@Override
protected Class<PartitionScanTask> taskJavaClass() {
return PartitionScanTask.class;
}
@Override
public NamedReference[] filterAttributes() {
Set<Integer> partitionFieldSourceIds = Sets.newHashSet();
for (PartitionSpec spec : specs()) {
for (PartitionField field : spec.fields()) {
partitionFieldSourceIds.add(field.sourceId());
}
}
Map<Integer, String> quotedNameById = SparkSchemaUtil.indexQuotedNameById(expectedSchema());
// the optimizer will look for an equality condition with filter attributes in a join
// as the scan has been already planned, filtering can only be done on projected attributes
// that's why only partition source fields that are part of the read schema can be reported
return partitionFieldSourceIds.stream()
.filter(fieldId -> expectedSchema().findField(fieldId) != null)
.map(fieldId -> Spark3Util.toNamedReference(quotedNameById.get(fieldId)))
.toArray(NamedReference[]::new);
}
@Override
public void filter(Filter[] filters) {
Expression runtimeFilterExpr = convertRuntimeFilters(filters);
if (runtimeFilterExpr != Expressions.alwaysTrue()) {
Map<Integer, Evaluator> evaluatorsBySpecId = Maps.newHashMap();
for (PartitionSpec spec : specs()) {
Expression inclusiveExpr =
Projections.inclusive(spec, caseSensitive()).project(runtimeFilterExpr);
Evaluator inclusive = new Evaluator(spec.partitionType(), inclusiveExpr);
evaluatorsBySpecId.put(spec.specId(), inclusive);
}
List<PartitionScanTask> filteredTasks =
tasks().stream()
.filter(
task -> {
Evaluator evaluator = evaluatorsBySpecId.get(task.spec().specId());
return evaluator.eval(task.partition());
})
.collect(Collectors.toList());
LOG.info(
"{} of {} task(s) for table {} matched runtime filter {}",
filteredTasks.size(),
tasks().size(),
table().name(),
ExpressionUtil.toSanitizedString(runtimeFilterExpr));
// don't invalidate tasks if the runtime filter had no effect to avoid planning splits again
if (filteredTasks.size() < tasks().size()) {
resetTasks(filteredTasks);
}
// save the evaluated filter for equals/hashCode
runtimeFilterExpressions.add(runtimeFilterExpr);
}
}
// at this moment, Spark can only pass IN filters for a single attribute
// if there are multiple filter attributes, Spark will pass two separate IN filters
private Expression convertRuntimeFilters(Filter[] filters) {
Expression runtimeFilterExpr = Expressions.alwaysTrue();
for (Filter filter : filters) {
Expression expr = SparkFilters.convert(filter);
if (expr != null) {
try {
Binder.bind(expectedSchema().asStruct(), expr, caseSensitive());
runtimeFilterExpr = Expressions.and(runtimeFilterExpr, expr);
} catch (ValidationException e) {
LOG.warn("Failed to bind {} to expected schema, skipping runtime filter", expr, e);
}
} else {
LOG.warn("Unsupported runtime filter {}", filter);
}
}
return runtimeFilterExpr;
}
@Override
public Statistics estimateStatistics() {
if (scan() == null) {
return estimateStatistics(null);
} else if (snapshotId != null) {
Snapshot snapshot = table().snapshot(snapshotId);
return estimateStatistics(snapshot);
} else if (asOfTimestamp != null) {
long snapshotIdAsOfTime = SnapshotUtil.snapshotIdAsOfTime(table(), asOfTimestamp);
Snapshot snapshot = table().snapshot(snapshotIdAsOfTime);
return estimateStatistics(snapshot);
} else if (branch() != null) {
Snapshot snapshot = table().snapshot(branch());
return estimateStatistics(snapshot);
} else if (tag != null) {
Snapshot snapshot = table().snapshot(tag);
return estimateStatistics(snapshot);
} else {
Snapshot snapshot = table().currentSnapshot();
return estimateStatistics(snapshot);
}
}
@Override
@SuppressWarnings("checkstyle:CyclomaticComplexity")
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SparkBatchQueryScan that = (SparkBatchQueryScan) o;
return table().name().equals(that.table().name())
&& Objects.equals(branch(), that.branch())
&& readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field ids
&& filterExpressions().toString().equals(that.filterExpressions().toString())
&& runtimeFilterExpressions.toString().equals(that.runtimeFilterExpressions.toString())
&& Objects.equals(snapshotId, that.snapshotId)
&& Objects.equals(startSnapshotId, that.startSnapshotId)
&& Objects.equals(endSnapshotId, that.endSnapshotId)
&& Objects.equals(asOfTimestamp, that.asOfTimestamp)
&& Objects.equals(tag, that.tag);
}
@Override
public int hashCode() {
return Objects.hash(
table().name(),
branch(),
readSchema(),
filterExpressions().toString(),
runtimeFilterExpressions.toString(),
snapshotId,
startSnapshotId,
endSnapshotId,
asOfTimestamp,
tag);
}
@Override
public String toString() {
return String.format(
"IcebergScan(table=%s, branch=%s, type=%s, filters=%s, runtimeFilters=%s, caseSensitive=%s)",
table(),
branch(),
expectedSchema().asStruct(),
filterExpressions(),
runtimeFilterExpressions,
caseSensitive());
}
}