forked from apache/iceberg
/
ScanContext.java
561 lines (487 loc) · 17.4 KB
/
ScanContext.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
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
/*
* 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.flink.source;
import java.io.Serializable;
import java.time.Duration;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.TimeUtils;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.flink.FlinkConfigOptions;
import org.apache.iceberg.flink.FlinkReadConf;
import org.apache.iceberg.flink.FlinkReadOptions;
/** Context object with optional arguments for a Flink Scan. */
@Internal
public class ScanContext implements Serializable {
private static final long serialVersionUID = 1L;
private final boolean caseSensitive;
private final boolean exposeLocality;
private final Long snapshotId;
private final String branch;
private final String tag;
private final StreamingStartingStrategy startingStrategy;
private final Long startSnapshotId;
private final Long startSnapshotTimestamp;
private final Long endSnapshotId;
private final Long asOfTimestamp;
private final String startTag;
private final String endTag;
private final Long splitSize;
private final Integer splitLookback;
private final Long splitOpenFileCost;
private final boolean isStreaming;
private final Duration monitorInterval;
private final String nameMapping;
private final Schema schema;
private final List<Expression> filters;
private final long limit;
private final boolean includeColumnStats;
private final Collection<String> includeStatsForColumns;
private final Integer planParallelism;
private final int maxPlanningSnapshotCount;
private final int maxAllowedPlanningFailures;
private ScanContext(
boolean caseSensitive,
Long snapshotId,
StreamingStartingStrategy startingStrategy,
Long startSnapshotTimestamp,
Long startSnapshotId,
Long endSnapshotId,
Long asOfTimestamp,
Long splitSize,
Integer splitLookback,
Long splitOpenFileCost,
boolean isStreaming,
Duration monitorInterval,
String nameMapping,
Schema schema,
List<Expression> filters,
long limit,
boolean includeColumnStats,
Collection<String> includeStatsForColumns,
boolean exposeLocality,
Integer planParallelism,
int maxPlanningSnapshotCount,
int maxAllowedPlanningFailures,
String branch,
String tag,
String startTag,
String endTag) {
this.caseSensitive = caseSensitive;
this.snapshotId = snapshotId;
this.tag = tag;
this.branch = branch;
this.startingStrategy = startingStrategy;
this.startSnapshotTimestamp = startSnapshotTimestamp;
this.startSnapshotId = startSnapshotId;
this.endSnapshotId = endSnapshotId;
this.asOfTimestamp = asOfTimestamp;
this.startTag = startTag;
this.endTag = endTag;
this.splitSize = splitSize;
this.splitLookback = splitLookback;
this.splitOpenFileCost = splitOpenFileCost;
this.isStreaming = isStreaming;
this.monitorInterval = monitorInterval;
this.nameMapping = nameMapping;
this.schema = schema;
this.filters = filters;
this.limit = limit;
this.includeColumnStats = includeColumnStats;
this.includeStatsForColumns = includeStatsForColumns;
this.exposeLocality = exposeLocality;
this.planParallelism = planParallelism;
this.maxPlanningSnapshotCount = maxPlanningSnapshotCount;
this.maxAllowedPlanningFailures = maxAllowedPlanningFailures;
validate();
}
private void validate() {
if (isStreaming) {
if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) {
Preconditions.checkArgument(
startSnapshotId != null,
"Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null");
Preconditions.checkArgument(
startSnapshotTimestamp == null,
"Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null");
}
if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) {
Preconditions.checkArgument(
startSnapshotTimestamp != null,
"Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null");
Preconditions.checkArgument(
startSnapshotId == null,
"Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null");
}
Preconditions.checkArgument(
branch == null,
String.format(
"Cannot scan table using ref %s configured for streaming reader yet", branch));
Preconditions.checkArgument(
tag == null,
String.format("Cannot scan table using ref %s configured for streaming reader", tag));
}
Preconditions.checkArgument(
!(startTag != null && startSnapshotId() != null),
"START_SNAPSHOT_ID and START_TAG cannot both be set.");
Preconditions.checkArgument(
!(endTag != null && endSnapshotId() != null),
"END_SNAPSHOT_ID and END_TAG cannot both be set.");
Preconditions.checkArgument(
maxAllowedPlanningFailures >= -1,
"Cannot set maxAllowedPlanningFailures to a negative number other than -1.");
}
public boolean caseSensitive() {
return caseSensitive;
}
public Long snapshotId() {
return snapshotId;
}
public String branch() {
return branch;
}
public String tag() {
return tag;
}
public String startTag() {
return startTag;
}
public String endTag() {
return endTag;
}
public StreamingStartingStrategy streamingStartingStrategy() {
return startingStrategy;
}
public Long startSnapshotTimestamp() {
return startSnapshotTimestamp;
}
public Long startSnapshotId() {
return startSnapshotId;
}
public Long endSnapshotId() {
return endSnapshotId;
}
public Long asOfTimestamp() {
return asOfTimestamp;
}
public Long splitSize() {
return splitSize;
}
public Integer splitLookback() {
return splitLookback;
}
public Long splitOpenFileCost() {
return splitOpenFileCost;
}
public boolean isStreaming() {
return isStreaming;
}
public Duration monitorInterval() {
return monitorInterval;
}
public String nameMapping() {
return nameMapping;
}
public Schema project() {
return schema;
}
public List<Expression> filters() {
return filters;
}
public long limit() {
return limit;
}
public boolean includeColumnStats() {
return includeColumnStats;
}
public Collection<String> includeStatsForColumns() {
return includeStatsForColumns;
}
public boolean exposeLocality() {
return exposeLocality;
}
public Integer planParallelism() {
return planParallelism;
}
public int maxPlanningSnapshotCount() {
return maxPlanningSnapshotCount;
}
public int maxAllowedPlanningFailures() {
return maxAllowedPlanningFailures;
}
public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) {
return ScanContext.builder()
.caseSensitive(caseSensitive)
.useSnapshotId(null)
.useBranch(branch)
.useTag(null)
.startSnapshotId(newStartSnapshotId)
.endSnapshotId(newEndSnapshotId)
.startTag(null)
.endTag(null)
.asOfTimestamp(null)
.splitSize(splitSize)
.splitLookback(splitLookback)
.splitOpenFileCost(splitOpenFileCost)
.streaming(isStreaming)
.monitorInterval(monitorInterval)
.nameMapping(nameMapping)
.project(schema)
.filters(filters)
.limit(limit)
.includeColumnStats(includeColumnStats)
.includeColumnStats(includeStatsForColumns)
.exposeLocality(exposeLocality)
.planParallelism(planParallelism)
.maxPlanningSnapshotCount(maxPlanningSnapshotCount)
.maxAllowedPlanningFailures(maxAllowedPlanningFailures)
.build();
}
public ScanContext copyWithSnapshotId(long newSnapshotId) {
return ScanContext.builder()
.caseSensitive(caseSensitive)
.useSnapshotId(newSnapshotId)
.useBranch(branch)
.useTag(tag)
.startSnapshotId(null)
.endSnapshotId(null)
.startTag(null)
.endTag(null)
.asOfTimestamp(null)
.splitSize(splitSize)
.splitLookback(splitLookback)
.splitOpenFileCost(splitOpenFileCost)
.streaming(isStreaming)
.monitorInterval(monitorInterval)
.nameMapping(nameMapping)
.project(schema)
.filters(filters)
.limit(limit)
.includeColumnStats(includeColumnStats)
.includeColumnStats(includeStatsForColumns)
.exposeLocality(exposeLocality)
.planParallelism(planParallelism)
.maxPlanningSnapshotCount(maxPlanningSnapshotCount)
.maxAllowedPlanningFailures(maxAllowedPlanningFailures)
.build();
}
public static Builder builder() {
return new Builder();
}
public static class Builder {
private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue();
private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue();
private String branch = FlinkReadOptions.BRANCH.defaultValue();
private String tag = FlinkReadOptions.TAG.defaultValue();
private String startTag = FlinkReadOptions.START_TAG.defaultValue();
private String endTag = FlinkReadOptions.END_TAG.defaultValue();
private StreamingStartingStrategy startingStrategy =
FlinkReadOptions.STARTING_STRATEGY_OPTION.defaultValue();
private Long startSnapshotTimestamp = FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.defaultValue();
private Long startSnapshotId = FlinkReadOptions.START_SNAPSHOT_ID.defaultValue();
private Long endSnapshotId = FlinkReadOptions.END_SNAPSHOT_ID.defaultValue();
private Long asOfTimestamp = FlinkReadOptions.AS_OF_TIMESTAMP.defaultValue();
private Long splitSize = FlinkReadOptions.SPLIT_SIZE_OPTION.defaultValue();
private Integer splitLookback = FlinkReadOptions.SPLIT_LOOKBACK_OPTION.defaultValue();
private Long splitOpenFileCost = FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION.defaultValue();
private boolean isStreaming = FlinkReadOptions.STREAMING_OPTION.defaultValue();
private Duration monitorInterval =
TimeUtils.parseDuration(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue());
private String nameMapping;
private Schema projectedSchema;
private List<Expression> filters;
private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue();
private boolean includeColumnStats =
FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue();
private Collection<String> includeStatsForColumns = null;
private boolean exposeLocality;
private Integer planParallelism =
FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue();
private int maxPlanningSnapshotCount =
FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue();
private int maxAllowedPlanningFailures =
FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue();
private Builder() {}
public Builder caseSensitive(boolean newCaseSensitive) {
this.caseSensitive = newCaseSensitive;
return this;
}
public Builder useSnapshotId(Long newSnapshotId) {
this.snapshotId = newSnapshotId;
return this;
}
public Builder useTag(String newTag) {
this.tag = newTag;
return this;
}
public Builder useBranch(String newBranch) {
this.branch = newBranch;
return this;
}
public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) {
this.startingStrategy = newStartingStrategy;
return this;
}
public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) {
this.startSnapshotTimestamp = newStartSnapshotTimestamp;
return this;
}
public Builder startSnapshotId(Long newStartSnapshotId) {
this.startSnapshotId = newStartSnapshotId;
return this;
}
public Builder endSnapshotId(Long newEndSnapshotId) {
this.endSnapshotId = newEndSnapshotId;
return this;
}
public Builder startTag(String newStartTag) {
this.startTag = newStartTag;
return this;
}
public Builder endTag(String newEndTag) {
this.endTag = newEndTag;
return this;
}
public Builder asOfTimestamp(Long newAsOfTimestamp) {
this.asOfTimestamp = newAsOfTimestamp;
return this;
}
public Builder splitSize(Long newSplitSize) {
this.splitSize = newSplitSize;
return this;
}
public Builder splitLookback(Integer newSplitLookback) {
this.splitLookback = newSplitLookback;
return this;
}
public Builder splitOpenFileCost(Long newSplitOpenFileCost) {
this.splitOpenFileCost = newSplitOpenFileCost;
return this;
}
public Builder streaming(boolean streaming) {
this.isStreaming = streaming;
return this;
}
public Builder monitorInterval(Duration newMonitorInterval) {
this.monitorInterval = newMonitorInterval;
return this;
}
public Builder nameMapping(String newNameMapping) {
this.nameMapping = newNameMapping;
return this;
}
public Builder project(Schema newProjectedSchema) {
this.projectedSchema = newProjectedSchema;
return this;
}
public Builder filters(List<Expression> newFilters) {
this.filters = newFilters;
return this;
}
public Builder limit(long newLimit) {
this.limit = newLimit;
return this;
}
public Builder includeColumnStats(boolean newIncludeColumnStats) {
this.includeColumnStats = newIncludeColumnStats;
return this;
}
public Builder includeColumnStats(Collection<String> newIncludeStatsForColumns) {
this.includeStatsForColumns = newIncludeStatsForColumns;
return this;
}
public Builder exposeLocality(boolean newExposeLocality) {
this.exposeLocality = newExposeLocality;
return this;
}
public Builder planParallelism(Integer parallelism) {
this.planParallelism = parallelism;
return this;
}
public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) {
this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount;
return this;
}
public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) {
this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures;
return this;
}
public Builder resolveConfig(
Table table, Map<String, String> readOptions, ReadableConfig readableConfig) {
FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig);
return this.useSnapshotId(flinkReadConf.snapshotId())
.useTag(flinkReadConf.tag())
.useBranch(flinkReadConf.branch())
.startTag(flinkReadConf.startTag())
.endTag(flinkReadConf.endTag())
.caseSensitive(flinkReadConf.caseSensitive())
.asOfTimestamp(flinkReadConf.asOfTimestamp())
.startingStrategy(flinkReadConf.startingStrategy())
.startSnapshotTimestamp(flinkReadConf.startSnapshotTimestamp())
.startSnapshotId(flinkReadConf.startSnapshotId())
.endSnapshotId(flinkReadConf.endSnapshotId())
.splitSize(flinkReadConf.splitSize())
.splitLookback(flinkReadConf.splitLookback())
.splitOpenFileCost(flinkReadConf.splitFileOpenCost())
.streaming(flinkReadConf.streaming())
.monitorInterval(flinkReadConf.monitorInterval())
.nameMapping(flinkReadConf.nameMapping())
.limit(flinkReadConf.limit())
.planParallelism(flinkReadConf.workerPoolSize())
.includeColumnStats(flinkReadConf.includeColumnStats())
.maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount())
.maxAllowedPlanningFailures(maxAllowedPlanningFailures);
}
public ScanContext build() {
return new ScanContext(
caseSensitive,
snapshotId,
startingStrategy,
startSnapshotTimestamp,
startSnapshotId,
endSnapshotId,
asOfTimestamp,
splitSize,
splitLookback,
splitOpenFileCost,
isStreaming,
monitorInterval,
nameMapping,
projectedSchema,
filters,
limit,
includeColumnStats,
includeStatsForColumns,
exposeLocality,
planParallelism,
maxPlanningSnapshotCount,
maxAllowedPlanningFailures,
branch,
tag,
startTag,
endTag);
}
}
}