-
Notifications
You must be signed in to change notification settings - Fork 13k
/
KafkaDynamicSource.java
619 lines (535 loc) · 25.8 KB
/
KafkaDynamicSource.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
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
/*
* 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.flink.streaming.connectors.kafka.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.connector.kafka.source.KafkaSource;
import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema.MetadataConverter;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.Projection;
import org.apache.flink.table.connector.ProviderContext;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown;
import org.apache.flink.table.data.GenericMapData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.data.TimestampData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.utils.DataTypeUtils;
import org.apache.flink.util.Preconditions;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Header;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
/** A version-agnostic Kafka {@link ScanTableSource}. */
@Internal
public class KafkaDynamicSource
implements ScanTableSource, SupportsReadingMetadata, SupportsWatermarkPushDown {
private static final String KAFKA_TRANSFORMATION = "kafka";
// --------------------------------------------------------------------------------------------
// Mutable attributes
// --------------------------------------------------------------------------------------------
/** Data type that describes the final output of the source. */
protected DataType producedDataType;
/** Metadata that is appended at the end of a physical source row. */
protected List<String> metadataKeys;
/** Watermark strategy that is used to generate per-partition watermark. */
protected @Nullable WatermarkStrategy<RowData> watermarkStrategy;
// --------------------------------------------------------------------------------------------
// Format attributes
// --------------------------------------------------------------------------------------------
private static final String VALUE_METADATA_PREFIX = "value.";
/** Data type to configure the formats. */
protected final DataType physicalDataType;
/** Optional format for decoding keys from Kafka. */
protected final @Nullable DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat;
/** Format for decoding values from Kafka. */
protected final DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat;
/** Indices that determine the key fields and the target position in the produced row. */
protected final int[] keyProjection;
/** Indices that determine the value fields and the target position in the produced row. */
protected final int[] valueProjection;
/** Prefix that needs to be removed from fields when constructing the physical data type. */
protected final @Nullable String keyPrefix;
// --------------------------------------------------------------------------------------------
// Kafka-specific attributes
// --------------------------------------------------------------------------------------------
/** The Kafka topics to consume. */
protected final List<String> topics;
/** The Kafka topic pattern to consume. */
protected final Pattern topicPattern;
/** Properties for the Kafka consumer. */
protected final Properties properties;
/**
* The startup mode for the contained consumer (default is {@link StartupMode#GROUP_OFFSETS}).
*/
protected final StartupMode startupMode;
/**
* Specific startup offsets; only relevant when startup mode is {@link
* StartupMode#SPECIFIC_OFFSETS}.
*/
protected final Map<KafkaTopicPartition, Long> specificStartupOffsets;
/**
* The start timestamp to locate partition offsets; only relevant when startup mode is {@link
* StartupMode#TIMESTAMP}.
*/
protected final long startupTimestampMillis;
/** Flag to determine source mode. In upsert mode, it will keep the tombstone message. * */
protected final boolean upsertMode;
protected final String tableIdentifier;
public KafkaDynamicSource(
DataType physicalDataType,
@Nullable DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat,
DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat,
int[] keyProjection,
int[] valueProjection,
@Nullable String keyPrefix,
@Nullable List<String> topics,
@Nullable Pattern topicPattern,
Properties properties,
StartupMode startupMode,
Map<KafkaTopicPartition, Long> specificStartupOffsets,
long startupTimestampMillis,
boolean upsertMode,
String tableIdentifier) {
// Format attributes
this.physicalDataType =
Preconditions.checkNotNull(
physicalDataType, "Physical data type must not be null.");
this.keyDecodingFormat = keyDecodingFormat;
this.valueDecodingFormat =
Preconditions.checkNotNull(
valueDecodingFormat, "Value decoding format must not be null.");
this.keyProjection =
Preconditions.checkNotNull(keyProjection, "Key projection must not be null.");
this.valueProjection =
Preconditions.checkNotNull(valueProjection, "Value projection must not be null.");
this.keyPrefix = keyPrefix;
// Mutable attributes
this.producedDataType = physicalDataType;
this.metadataKeys = Collections.emptyList();
this.watermarkStrategy = null;
// Kafka-specific attributes
Preconditions.checkArgument(
(topics != null && topicPattern == null)
|| (topics == null && topicPattern != null),
"Either Topic or Topic Pattern must be set for source.");
this.topics = topics;
this.topicPattern = topicPattern;
this.properties = Preconditions.checkNotNull(properties, "Properties must not be null.");
this.startupMode =
Preconditions.checkNotNull(startupMode, "Startup mode must not be null.");
this.specificStartupOffsets =
Preconditions.checkNotNull(
specificStartupOffsets, "Specific offsets must not be null.");
this.startupTimestampMillis = startupTimestampMillis;
this.upsertMode = upsertMode;
this.tableIdentifier = tableIdentifier;
}
@Override
public ChangelogMode getChangelogMode() {
return valueDecodingFormat.getChangelogMode();
}
@Override
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) {
final DeserializationSchema<RowData> keyDeserialization =
createDeserialization(context, keyDecodingFormat, keyProjection, keyPrefix);
final DeserializationSchema<RowData> valueDeserialization =
createDeserialization(context, valueDecodingFormat, valueProjection, null);
final TypeInformation<RowData> producedTypeInfo =
context.createTypeInformation(producedDataType);
final KafkaSource<RowData> kafkaSource =
createKafkaSource(keyDeserialization, valueDeserialization, producedTypeInfo);
return new DataStreamScanProvider() {
@Override
public DataStream<RowData> produceDataStream(
ProviderContext providerContext, StreamExecutionEnvironment execEnv) {
if (watermarkStrategy == null) {
watermarkStrategy = WatermarkStrategy.noWatermarks();
}
DataStreamSource<RowData> sourceStream =
execEnv.fromSource(
kafkaSource, watermarkStrategy, "KafkaSource-" + tableIdentifier);
providerContext.generateUid(KAFKA_TRANSFORMATION).ifPresent(sourceStream::uid);
return sourceStream;
}
@Override
public boolean isBounded() {
return kafkaSource.getBoundedness() == Boundedness.BOUNDED;
}
};
}
@Override
public Map<String, DataType> listReadableMetadata() {
final Map<String, DataType> metadataMap = new LinkedHashMap<>();
// according to convention, the order of the final row must be
// PHYSICAL + FORMAT METADATA + CONNECTOR METADATA
// where the format metadata has highest precedence
// add value format metadata with prefix
valueDecodingFormat
.listReadableMetadata()
.forEach((key, value) -> metadataMap.put(VALUE_METADATA_PREFIX + key, value));
// add connector metadata
Stream.of(ReadableMetadata.values())
.forEachOrdered(m -> metadataMap.putIfAbsent(m.key, m.dataType));
return metadataMap;
}
@Override
public void applyReadableMetadata(List<String> metadataKeys, DataType producedDataType) {
// separate connector and format metadata
final List<String> formatMetadataKeys =
metadataKeys.stream()
.filter(k -> k.startsWith(VALUE_METADATA_PREFIX))
.collect(Collectors.toList());
final List<String> connectorMetadataKeys = new ArrayList<>(metadataKeys);
connectorMetadataKeys.removeAll(formatMetadataKeys);
// push down format metadata
final Map<String, DataType> formatMetadata = valueDecodingFormat.listReadableMetadata();
if (formatMetadata.size() > 0) {
final List<String> requestedFormatMetadataKeys =
formatMetadataKeys.stream()
.map(k -> k.substring(VALUE_METADATA_PREFIX.length()))
.collect(Collectors.toList());
valueDecodingFormat.applyReadableMetadata(requestedFormatMetadataKeys);
}
this.metadataKeys = connectorMetadataKeys;
this.producedDataType = producedDataType;
}
@Override
public boolean supportsMetadataProjection() {
return false;
}
@Override
public void applyWatermark(WatermarkStrategy<RowData> watermarkStrategy) {
this.watermarkStrategy = watermarkStrategy;
}
@Override
public DynamicTableSource copy() {
final KafkaDynamicSource copy =
new KafkaDynamicSource(
physicalDataType,
keyDecodingFormat,
valueDecodingFormat,
keyProjection,
valueProjection,
keyPrefix,
topics,
topicPattern,
properties,
startupMode,
specificStartupOffsets,
startupTimestampMillis,
upsertMode,
tableIdentifier);
copy.producedDataType = producedDataType;
copy.metadataKeys = metadataKeys;
copy.watermarkStrategy = watermarkStrategy;
return copy;
}
@Override
public String asSummaryString() {
return "Kafka table source";
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final KafkaDynamicSource that = (KafkaDynamicSource) o;
return Objects.equals(producedDataType, that.producedDataType)
&& Objects.equals(metadataKeys, that.metadataKeys)
&& Objects.equals(physicalDataType, that.physicalDataType)
&& Objects.equals(keyDecodingFormat, that.keyDecodingFormat)
&& Objects.equals(valueDecodingFormat, that.valueDecodingFormat)
&& Arrays.equals(keyProjection, that.keyProjection)
&& Arrays.equals(valueProjection, that.valueProjection)
&& Objects.equals(keyPrefix, that.keyPrefix)
&& Objects.equals(topics, that.topics)
&& Objects.equals(String.valueOf(topicPattern), String.valueOf(that.topicPattern))
&& Objects.equals(properties, that.properties)
&& startupMode == that.startupMode
&& Objects.equals(specificStartupOffsets, that.specificStartupOffsets)
&& startupTimestampMillis == that.startupTimestampMillis
&& Objects.equals(upsertMode, that.upsertMode)
&& Objects.equals(tableIdentifier, that.tableIdentifier)
&& Objects.equals(watermarkStrategy, that.watermarkStrategy);
}
@Override
public int hashCode() {
return Objects.hash(
producedDataType,
metadataKeys,
physicalDataType,
keyDecodingFormat,
valueDecodingFormat,
keyProjection,
valueProjection,
keyPrefix,
topics,
topicPattern,
properties,
startupMode,
specificStartupOffsets,
startupTimestampMillis,
upsertMode,
tableIdentifier,
watermarkStrategy);
}
// --------------------------------------------------------------------------------------------
protected KafkaSource<RowData> createKafkaSource(
DeserializationSchema<RowData> keyDeserialization,
DeserializationSchema<RowData> valueDeserialization,
TypeInformation<RowData> producedTypeInfo) {
final KafkaDeserializationSchema<RowData> kafkaDeserializer =
createKafkaDeserializationSchema(
keyDeserialization, valueDeserialization, producedTypeInfo);
final KafkaSourceBuilder<RowData> kafkaSourceBuilder = KafkaSource.builder();
if (topics != null) {
kafkaSourceBuilder.setTopics(topics);
} else {
kafkaSourceBuilder.setTopicPattern(topicPattern);
}
switch (startupMode) {
case EARLIEST:
kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.earliest());
break;
case LATEST:
kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.latest());
break;
case GROUP_OFFSETS:
String offsetResetConfig =
properties.getProperty(
ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
OffsetResetStrategy.NONE.name());
OffsetResetStrategy offsetResetStrategy = getResetStrategy(offsetResetConfig);
kafkaSourceBuilder.setStartingOffsets(
OffsetsInitializer.committedOffsets(offsetResetStrategy));
break;
case SPECIFIC_OFFSETS:
Map<TopicPartition, Long> offsets = new HashMap<>();
specificStartupOffsets.forEach(
(tp, offset) ->
offsets.put(
new TopicPartition(tp.getTopic(), tp.getPartition()),
offset));
kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.offsets(offsets));
break;
case TIMESTAMP:
kafkaSourceBuilder.setStartingOffsets(
OffsetsInitializer.timestamp(startupTimestampMillis));
break;
}
kafkaSourceBuilder
.setProperties(properties)
.setDeserializer(KafkaRecordDeserializationSchema.of(kafkaDeserializer));
return kafkaSourceBuilder.build();
}
private OffsetResetStrategy getResetStrategy(String offsetResetConfig) {
return Arrays.stream(OffsetResetStrategy.values())
.filter(ors -> ors.name().equals(offsetResetConfig.toUpperCase(Locale.ROOT)))
.findAny()
.orElseThrow(
() ->
new IllegalArgumentException(
String.format(
"%s can not be set to %s. Valid values: [%s]",
ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
offsetResetConfig,
Arrays.stream(OffsetResetStrategy.values())
.map(Enum::name)
.map(String::toLowerCase)
.collect(Collectors.joining(",")))));
}
private KafkaDeserializationSchema<RowData> createKafkaDeserializationSchema(
DeserializationSchema<RowData> keyDeserialization,
DeserializationSchema<RowData> valueDeserialization,
TypeInformation<RowData> producedTypeInfo) {
final MetadataConverter[] metadataConverters =
metadataKeys.stream()
.map(
k ->
Stream.of(ReadableMetadata.values())
.filter(rm -> rm.key.equals(k))
.findFirst()
.orElseThrow(IllegalStateException::new))
.map(m -> m.converter)
.toArray(MetadataConverter[]::new);
// check if connector metadata is used at all
final boolean hasMetadata = metadataKeys.size() > 0;
// adjust physical arity with value format's metadata
final int adjustedPhysicalArity =
DataType.getFieldDataTypes(producedDataType).size() - metadataKeys.size();
// adjust value format projection to include value format's metadata columns at the end
final int[] adjustedValueProjection =
IntStream.concat(
IntStream.of(valueProjection),
IntStream.range(
keyProjection.length + valueProjection.length,
adjustedPhysicalArity))
.toArray();
return new DynamicKafkaDeserializationSchema(
adjustedPhysicalArity,
keyDeserialization,
keyProjection,
valueDeserialization,
adjustedValueProjection,
hasMetadata,
metadataConverters,
producedTypeInfo,
upsertMode);
}
private @Nullable DeserializationSchema<RowData> createDeserialization(
DynamicTableSource.Context context,
@Nullable DecodingFormat<DeserializationSchema<RowData>> format,
int[] projection,
@Nullable String prefix) {
if (format == null) {
return null;
}
DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType);
if (prefix != null) {
physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix);
}
return format.createRuntimeDecoder(context, physicalFormatDataType);
}
// --------------------------------------------------------------------------------------------
// Metadata handling
// --------------------------------------------------------------------------------------------
enum ReadableMetadata {
TOPIC(
"topic",
DataTypes.STRING().notNull(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(ConsumerRecord<?, ?> record) {
return StringData.fromString(record.topic());
}
}),
PARTITION(
"partition",
DataTypes.INT().notNull(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(ConsumerRecord<?, ?> record) {
return record.partition();
}
}),
HEADERS(
"headers",
// key and value of the map are nullable to make handling easier in queries
DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.BYTES().nullable())
.notNull(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(ConsumerRecord<?, ?> record) {
final Map<StringData, byte[]> map = new HashMap<>();
for (Header header : record.headers()) {
map.put(StringData.fromString(header.key()), header.value());
}
return new GenericMapData(map);
}
}),
LEADER_EPOCH(
"leader-epoch",
DataTypes.INT().nullable(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(ConsumerRecord<?, ?> record) {
return record.leaderEpoch().orElse(null);
}
}),
OFFSET(
"offset",
DataTypes.BIGINT().notNull(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(ConsumerRecord<?, ?> record) {
return record.offset();
}
}),
TIMESTAMP(
"timestamp",
DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(ConsumerRecord<?, ?> record) {
return TimestampData.fromEpochMillis(record.timestamp());
}
}),
TIMESTAMP_TYPE(
"timestamp-type",
DataTypes.STRING().notNull(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(ConsumerRecord<?, ?> record) {
return StringData.fromString(record.timestampType().toString());
}
});
final String key;
final DataType dataType;
final MetadataConverter converter;
ReadableMetadata(String key, DataType dataType, MetadataConverter converter) {
this.key = key;
this.dataType = dataType;
this.converter = converter;
}
}
}