forked from apache/iceberg
/
BaseScan.java
291 lines (246 loc) · 8.69 KB
/
BaseScan.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
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
/*
* 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;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.stream.Collectors;
import org.apache.iceberg.expressions.Binder;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.metrics.MetricsReporter;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.util.PropertyUtil;
abstract class BaseScan<ThisT, T extends ScanTask, G extends ScanTaskGroup<T>>
implements Scan<ThisT, T, G> {
protected static final List<String> SCAN_COLUMNS =
ImmutableList.of(
"snapshot_id",
"file_path",
"file_ordinal",
"file_format",
"block_size_in_bytes",
"file_size_in_bytes",
"record_count",
"partition",
"key_metadata",
"split_offsets");
private static final List<String> STATS_COLUMNS =
ImmutableList.of(
"value_counts",
"null_value_counts",
"nan_value_counts",
"lower_bounds",
"upper_bounds",
"column_sizes");
protected static final List<String> SCAN_WITH_STATS_COLUMNS =
ImmutableList.<String>builder().addAll(SCAN_COLUMNS).addAll(STATS_COLUMNS).build();
protected static final List<String> DELETE_SCAN_COLUMNS =
ImmutableList.of(
"snapshot_id",
"content",
"file_path",
"file_ordinal",
"file_format",
"block_size_in_bytes",
"file_size_in_bytes",
"record_count",
"partition",
"key_metadata",
"split_offsets",
"equality_ids");
protected static final List<String> DELETE_SCAN_WITH_STATS_COLUMNS =
ImmutableList.<String>builder().addAll(DELETE_SCAN_COLUMNS).addAll(STATS_COLUMNS).build();
protected static final boolean PLAN_SCANS_WITH_WORKER_POOL =
SystemConfigs.SCAN_THREAD_POOL_ENABLED.value();
private final Table table;
private final Schema schema;
private final TableScanContext context;
protected BaseScan(Table table, Schema schema, TableScanContext context) {
this.table = table;
this.schema = schema;
this.context = context;
}
public Table table() {
return table;
}
protected FileIO io() {
return table.io();
}
protected Schema tableSchema() {
return schema;
}
protected TableScanContext context() {
return context;
}
protected Map<String, String> options() {
return context().options();
}
protected List<String> scanColumns() {
return context.returnColumnStats() ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS;
}
protected boolean shouldReturnColumnStats() {
return context().returnColumnStats();
}
protected Set<Integer> columnsToKeepStats() {
return context().columnsToKeepStats();
}
protected boolean shouldIgnoreResiduals() {
return context().ignoreResiduals();
}
protected Expression residualFilter() {
return shouldIgnoreResiduals() ? Expressions.alwaysTrue() : filter();
}
protected boolean shouldPlanWithExecutor() {
return PLAN_SCANS_WITH_WORKER_POOL || context().planWithCustomizedExecutor();
}
protected ExecutorService planExecutor() {
return context().planExecutor();
}
protected abstract ThisT newRefinedScan(
Table newTable, Schema newSchema, TableScanContext newContext);
@Override
public ThisT option(String property, String value) {
return newRefinedScan(table, schema, context.withOption(property, value));
}
@Override
public ThisT project(Schema projectedSchema) {
return newRefinedScan(table, schema, context.project(projectedSchema));
}
@Override
public ThisT caseSensitive(boolean caseSensitive) {
return newRefinedScan(table, schema, context.setCaseSensitive(caseSensitive));
}
@Override
public boolean isCaseSensitive() {
return context().caseSensitive();
}
@Override
public ThisT includeColumnStats() {
return newRefinedScan(table, schema, context.shouldReturnColumnStats(true));
}
@Override
public ThisT includeColumnStats(Collection<String> requestedColumns) {
return newRefinedScan(
table,
schema,
context
.shouldReturnColumnStats(true)
.columnsToKeepStats(
requestedColumns.stream()
.map(c -> schema.findField(c).fieldId())
.collect(Collectors.toSet())));
}
@Override
public ThisT select(Collection<String> columns) {
return newRefinedScan(table, schema, context.selectColumns(columns));
}
@Override
public ThisT filter(Expression expr) {
return newRefinedScan(
table, schema, context.filterRows(Expressions.and(context.rowFilter(), expr)));
}
@Override
public Expression filter() {
return context().rowFilter();
}
@Override
public ThisT ignoreResiduals() {
return newRefinedScan(table, schema, context.ignoreResiduals(true));
}
@Override
public ThisT planWith(ExecutorService executorService) {
return newRefinedScan(table, schema, context.planWith(executorService));
}
@Override
public Schema schema() {
return lazyColumnProjection(context, schema);
}
@Override
public long targetSplitSize() {
long tableValue =
PropertyUtil.propertyAsLong(
table().properties(), TableProperties.SPLIT_SIZE, TableProperties.SPLIT_SIZE_DEFAULT);
return PropertyUtil.propertyAsLong(context.options(), TableProperties.SPLIT_SIZE, tableValue);
}
@Override
public int splitLookback() {
int tableValue =
PropertyUtil.propertyAsInt(
table().properties(),
TableProperties.SPLIT_LOOKBACK,
TableProperties.SPLIT_LOOKBACK_DEFAULT);
return PropertyUtil.propertyAsInt(
context.options(), TableProperties.SPLIT_LOOKBACK, tableValue);
}
@Override
public long splitOpenFileCost() {
long tableValue =
PropertyUtil.propertyAsLong(
table().properties(),
TableProperties.SPLIT_OPEN_FILE_COST,
TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
return PropertyUtil.propertyAsLong(
context.options(), TableProperties.SPLIT_OPEN_FILE_COST, tableValue);
}
/**
* Resolve the schema to be projected lazily.
*
* <p>If there are selected columns from scan context, selected columns are projected to the table
* schema. Otherwise, projected schema from scan context shall be returned.
*
* @param context scan context
* @param schema table schema
* @return the Schema to project
*/
private static Schema lazyColumnProjection(TableScanContext context, Schema schema) {
Collection<String> selectedColumns = context.selectedColumns();
if (selectedColumns != null) {
Set<Integer> requiredFieldIds = Sets.newHashSet();
// all of the filter columns are required
requiredFieldIds.addAll(
Binder.boundReferences(
schema.asStruct(),
Collections.singletonList(context.rowFilter()),
context.caseSensitive()));
// all of the projection columns are required
Set<Integer> selectedIds;
if (context.caseSensitive()) {
selectedIds = TypeUtil.getProjectedIds(schema.select(selectedColumns));
} else {
selectedIds = TypeUtil.getProjectedIds(schema.caseInsensitiveSelect(selectedColumns));
}
requiredFieldIds.addAll(selectedIds);
return TypeUtil.project(schema, requiredFieldIds);
} else if (context.projectedSchema() != null) {
return context.projectedSchema();
}
return schema;
}
@Override
public ThisT metricsReporter(MetricsReporter reporter) {
return newRefinedScan(table, schema, context.reportWith(reporter));
}
}