Skip to content

Commit c7faaca

Browse files
committed
[FLINK-25509][connector/kafka] Support to use RecordEvaluator to filter records in kafka connector
1 parent 0fed445 commit c7faaca

21 files changed

+654
-65
lines changed

flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import org.apache.flink.api.connector.source.SplitEnumerator;
3030
import org.apache.flink.api.connector.source.SplitEnumeratorContext;
3131
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
32+
import org.apache.flink.connector.base.source.reader.RecordEvaluator;
3233
import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService;
3334
import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState;
3435
import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumStateSerializer;
@@ -42,6 +43,8 @@
4243
import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
4344
import org.apache.flink.core.io.SimpleVersionedSerializer;
4445

46+
import javax.annotation.Nullable;
47+
4548
import java.util.Properties;
4649

4750
/**
@@ -87,6 +90,7 @@ public class DynamicKafkaSource<T>
8790
private final OffsetsInitializer stoppingOffsetsInitializer;
8891
private final Properties properties;
8992
private final Boundedness boundedness;
93+
@Nullable private final RecordEvaluator<T> eofRecordEvaluator;
9094

9195
DynamicKafkaSource(
9296
KafkaStreamSubscriber kafkaStreamSubscriber,
@@ -95,14 +99,16 @@ public class DynamicKafkaSource<T>
9599
OffsetsInitializer startingOffsetsInitializer,
96100
OffsetsInitializer stoppingOffsetsInitializer,
97101
Properties properties,
98-
Boundedness boundedness) {
102+
Boundedness boundedness,
103+
@Nullable RecordEvaluator<T> eofRecordEvaluator) {
99104
this.kafkaStreamSubscriber = kafkaStreamSubscriber;
100105
this.deserializationSchema = deserializationSchema;
101106
this.properties = properties;
102107
this.kafkaMetadataService = kafkaMetadataService;
103108
this.startingOffsetsInitializer = startingOffsetsInitializer;
104109
this.stoppingOffsetsInitializer = stoppingOffsetsInitializer;
105110
this.boundedness = boundedness;
111+
this.eofRecordEvaluator = eofRecordEvaluator;
106112
}
107113

108114
/**
@@ -134,7 +140,8 @@ public Boundedness getBoundedness() {
134140
@Override
135141
public SourceReader<T, DynamicKafkaSourceSplit> createReader(
136142
SourceReaderContext readerContext) {
137-
return new DynamicKafkaSourceReader<>(readerContext, deserializationSchema, properties);
143+
return new DynamicKafkaSourceReader<>(
144+
readerContext, deserializationSchema, properties, eofRecordEvaluator);
138145
}
139146

140147
/**

flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020

2121
import org.apache.flink.annotation.Experimental;
2222
import org.apache.flink.api.connector.source.Boundedness;
23+
import org.apache.flink.connector.base.source.reader.RecordEvaluator;
2324
import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService;
2425
import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber;
2526
import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber;
@@ -52,6 +53,7 @@ public class DynamicKafkaSourceBuilder<T> {
5253
private OffsetsInitializer stoppingOffsetsInitializer;
5354
private Boundedness boundedness;
5455
private final Properties props;
56+
private RecordEvaluator<T> eofRecordEvaluator;
5557

5658
DynamicKafkaSourceBuilder() {
5759
this.kafkaStreamSubscriber = null;
@@ -140,6 +142,18 @@ public DynamicKafkaSourceBuilder<T> setDeserializer(
140142
return this;
141143
}
142144

145+
/**
146+
* Set the {@link RecordEvaluator}.
147+
*
148+
* @param eofRecordEvaluator the {@link RecordEvaluator}.
149+
* @return the builder.
150+
*/
151+
public DynamicKafkaSourceBuilder<T> setEofRecordEvaluator(
152+
RecordEvaluator<T> eofRecordEvaluator) {
153+
this.eofRecordEvaluator = eofRecordEvaluator;
154+
return this;
155+
}
156+
143157
/**
144158
* Set the starting offsets of the stream. This will be applied to all clusters.
145159
*
@@ -217,7 +231,8 @@ public DynamicKafkaSource<T> build() {
217231
startingOffsetsInitializer,
218232
stoppingOffsetsInitializer,
219233
props,
220-
boundedness);
234+
boundedness,
235+
eofRecordEvaluator);
221236
}
222237

223238
// Below are utility methods, code and structure are mostly copied over from KafkaSourceBuilder

flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import org.apache.flink.api.connector.source.SourceReader;
2727
import org.apache.flink.api.connector.source.SourceReaderContext;
2828
import org.apache.flink.configuration.Configuration;
29+
import org.apache.flink.connector.base.source.reader.RecordEvaluator;
2930
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
3031
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
3132
import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata;
@@ -54,6 +55,8 @@
5455
import org.slf4j.Logger;
5556
import org.slf4j.LoggerFactory;
5657

58+
import javax.annotation.Nullable;
59+
5760
import java.util.ArrayList;
5861
import java.util.HashMap;
5962
import java.util.HashSet;
@@ -95,11 +98,13 @@ public class DynamicKafkaSourceReader<T> implements SourceReader<T, DynamicKafka
9598
private boolean isActivelyConsumingSplits;
9699
private boolean isNoMoreSplits;
97100
private AtomicBoolean restartingReaders;
101+
@Nullable private final RecordEvaluator<T> eofRecordEvaluator;
98102

99103
public DynamicKafkaSourceReader(
100104
SourceReaderContext readerContext,
101105
KafkaRecordDeserializationSchema<T> deserializationSchema,
102-
Properties properties) {
106+
Properties properties,
107+
@Nullable RecordEvaluator<T> eofRecordEvaluator) {
103108
this.readerContext = readerContext;
104109
this.clusterReaderMap = new TreeMap<>();
105110
this.deserializationSchema = deserializationSchema;
@@ -116,6 +121,7 @@ public DynamicKafkaSourceReader(
116121
this.isActivelyConsumingSplits = false;
117122
this.restartingReaders = new AtomicBoolean();
118123
this.clustersProperties = new HashMap<>();
124+
this.eofRecordEvaluator = eofRecordEvaluator;
119125
}
120126

121127
/**
@@ -448,7 +454,8 @@ public UserCodeClassLoader getUserCodeClassLoader() {
448454
}
449455
});
450456

451-
KafkaRecordEmitter<T> recordEmitter = new KafkaRecordEmitter<>(deserializationSchema);
457+
KafkaRecordEmitter<T> recordEmitter =
458+
new KafkaRecordEmitter<>(deserializationSchema, eofRecordEvaluator);
452459
return new KafkaSourceReader<>(
453460
elementsQueue,
454461
new KafkaSourceFetcherManager(
@@ -463,7 +470,8 @@ public UserCodeClassLoader getUserCodeClassLoader() {
463470
recordEmitter,
464471
toConfiguration(readerSpecificProperties),
465472
readerContext,
466-
kafkaSourceReaderMetrics);
473+
kafkaSourceReaderMetrics,
474+
eofRecordEvaluator);
467475
}
468476

469477
/**

flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java

Lines changed: 14 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.apache.flink.api.connector.source.SplitEnumeratorContext;
3232
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
3333
import org.apache.flink.configuration.Configuration;
34+
import org.apache.flink.connector.base.source.reader.RecordEvaluator;
3435
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
3536
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
3637
import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState;
@@ -102,11 +103,13 @@ public class KafkaSource<OUT>
102103
private final Properties props;
103104
// Client rackId callback
104105
private final SerializableSupplier<String> rackIdSupplier;
106+
@Nullable private RecordEvaluator<OUT> eofRecordEvaluator;
105107

106108
KafkaSource(
107109
KafkaSubscriber subscriber,
108110
OffsetsInitializer startingOffsetsInitializer,
109111
@Nullable OffsetsInitializer stoppingOffsetsInitializer,
112+
@Nullable RecordEvaluator<OUT> eofRecordEvaluator,
110113
Boundedness boundedness,
111114
KafkaRecordDeserializationSchema<OUT> deserializationSchema,
112115
Properties props,
@@ -118,6 +121,7 @@ public class KafkaSource<OUT>
118121
this.deserializationSchema = deserializationSchema;
119122
this.props = props;
120123
this.rackIdSupplier = rackIdSupplier;
124+
this.eofRecordEvaluator = eofRecordEvaluator;
121125
}
122126

123127
/**
@@ -171,7 +175,8 @@ public UserCodeClassLoader getUserCodeClassLoader() {
171175
Optional.ofNullable(rackIdSupplier)
172176
.map(Supplier::get)
173177
.orElse(null));
174-
KafkaRecordEmitter<OUT> recordEmitter = new KafkaRecordEmitter<>(deserializationSchema);
178+
KafkaRecordEmitter<OUT> recordEmitter =
179+
new KafkaRecordEmitter<>(deserializationSchema, eofRecordEvaluator);
175180

176181
return new KafkaSourceReader<>(
177182
elementsQueue,
@@ -180,7 +185,8 @@ public UserCodeClassLoader getUserCodeClassLoader() {
180185
recordEmitter,
181186
toConfiguration(props),
182187
readerContext,
183-
kafkaSourceReaderMetrics);
188+
kafkaSourceReaderMetrics,
189+
eofRecordEvaluator);
184190
}
185191

186192
@Internal
@@ -251,4 +257,10 @@ KafkaSubscriber getKafkaSubscriber() {
251257
OffsetsInitializer getStoppingOffsetsInitializer() {
252258
return stoppingOffsetsInitializer;
253259
}
260+
261+
@VisibleForTesting
262+
@Nullable
263+
RecordEvaluator<OUT> getEofRecordEvaluator() {
264+
return eofRecordEvaluator;
265+
}
254266
}

flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import org.apache.flink.annotation.PublicEvolving;
2222
import org.apache.flink.api.common.serialization.DeserializationSchema;
2323
import org.apache.flink.api.connector.source.Boundedness;
24+
import org.apache.flink.connector.base.source.reader.RecordEvaluator;
2425
import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer;
2526
import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
2627
import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializerValidator;
@@ -107,6 +108,7 @@ public class KafkaSourceBuilder<OUT> {
107108
protected Properties props;
108109
// Client rackId supplier
109110
private SerializableSupplier<String> rackIdSupplier;
111+
private RecordEvaluator<OUT> eofRecordEvaluator;
110112

111113
KafkaSourceBuilder() {
112114
this.subscriber = null;
@@ -353,6 +355,26 @@ public KafkaSourceBuilder<OUT> setValueOnlyDeserializer(
353355
return this;
354356
}
355357

358+
/**
359+
* Sets the optional {@link RecordEvaluator eofRecordEvaluator} for KafkaSource.
360+
*
361+
* <p>When the evaluator is specified, it is invoked for each de-serialized record to determine
362+
* whether the corresponding split has reached end of stream. If a record is matched by the
363+
* evaluator, the source would not emit this record as well as the following records in the same
364+
* split.
365+
*
366+
* <p>Note that the evaluator works jointly with the stopping offsets specified by the {@link
367+
* #setBounded(OffsetsInitializer)} or the {@link #setUnbounded(OffsetsInitializer)}. The source
368+
* stops consuming from a split when any of these conditions is met.
369+
*
370+
* @param eofRecordEvaluator a {@link RecordEvaluator recordEvaluator}
371+
* @return this KafkaSourceBuilder.
372+
*/
373+
public KafkaSourceBuilder<OUT> setEofRecordEvaluator(RecordEvaluator<OUT> eofRecordEvaluator) {
374+
this.eofRecordEvaluator = eofRecordEvaluator;
375+
return this;
376+
}
377+
356378
/**
357379
* Sets the client id prefix of this KafkaSource.
358380
*
@@ -435,6 +457,7 @@ public KafkaSource<OUT> build() {
435457
subscriber,
436458
startingOffsetsInitializer,
437459
stoppingOffsetsInitializer,
460+
eofRecordEvaluator,
438461
boundedness,
439462
deserializationSchema,
440463
props,

flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java

Lines changed: 44 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
2626
import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
2727
import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
28+
import org.apache.flink.connector.base.source.reader.splitreader.SplitsRemoval;
2829
import org.apache.flink.connector.kafka.source.KafkaSourceOptions;
2930
import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics;
3031
import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
@@ -76,6 +77,9 @@ public class KafkaPartitionSplitReader
7677
// Tracking empty splits that has not been added to finished splits in fetch()
7778
private final Set<String> emptySplits = new HashSet<>();
7879

80+
// Tracking removed splits that has not been added to finished splits in fetch()
81+
private final Set<String> removedSplits = new HashSet<>();
82+
7983
public KafkaPartitionSplitReader(
8084
Properties props,
8185
SourceReaderContext context,
@@ -116,7 +120,7 @@ public RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> fetch() throws IOExce
116120
KafkaPartitionSplitRecords recordsBySplits =
117121
new KafkaPartitionSplitRecords(
118122
ConsumerRecords.empty(), kafkaSourceReaderMetrics);
119-
markEmptySplitsAsFinished(recordsBySplits);
123+
markSplitsAsFinished(recordsBySplits);
120124
return recordsBySplits;
121125
}
122126
KafkaPartitionSplitRecords recordsBySplits =
@@ -148,7 +152,7 @@ public RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> fetch() throws IOExce
148152
kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, trackTp);
149153
});
150154

151-
markEmptySplitsAsFinished(recordsBySplits);
155+
markSplitsAsFinished(recordsBySplits);
152156

153157
// Unassign the partitions that has finished.
154158
if (!finishedPartitions.isEmpty()) {
@@ -162,25 +166,55 @@ public RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> fetch() throws IOExce
162166
return recordsBySplits;
163167
}
164168

165-
private void markEmptySplitsAsFinished(KafkaPartitionSplitRecords recordsBySplits) {
169+
private void markSplitsAsFinished(KafkaPartitionSplitRecords recordsBySplits) {
170+
// Some splits are discovered as empty when handling split additions. These splits should be
171+
// added to finished splits to clean up states in split fetcher and source reader.
172+
markSplitsAsFinished(emptySplits, recordsBySplits);
173+
174+
// Some splits are removed when handling split changes. These splits should be
175+
// added to finished splits to clean up states in split fetcher and source reader.
176+
markSplitsAsFinished(removedSplits, recordsBySplits);
177+
}
178+
179+
private void markSplitsAsFinished(
180+
Set<String> splits, KafkaPartitionSplitRecords recordsBySplits) {
166181
// Some splits are discovered as empty when handling split additions. These splits should be
167182
// added to finished splits to clean up states in split fetcher and source reader.
168-
if (!emptySplits.isEmpty()) {
169-
recordsBySplits.finishedSplits.addAll(emptySplits);
170-
emptySplits.clear();
183+
if (!splits.isEmpty()) {
184+
recordsBySplits.finishedSplits.addAll(splits);
185+
splits.clear();
171186
}
172187
}
173188

174189
@Override
175190
public void handleSplitsChanges(SplitsChange<KafkaPartitionSplit> splitsChange) {
176-
// Get all the partition assignments and stopping offsets.
177-
if (!(splitsChange instanceof SplitsAddition)) {
191+
if (splitsChange instanceof SplitsAddition) {
192+
// Get all the partition assignments and stopping offsets.
193+
handleSplitsAddition(splitsChange);
194+
} else if (splitsChange instanceof SplitsRemoval) {
195+
handleSplitsRemoval(splitsChange);
196+
} else {
178197
throw new UnsupportedOperationException(
179198
String.format(
180199
"The SplitChange type of %s is not supported.",
181200
splitsChange.getClass()));
182201
}
202+
}
203+
204+
private void handleSplitsRemoval(SplitsChange<KafkaPartitionSplit> splitsRemoval) {
205+
removedSplits.addAll(
206+
splitsRemoval.splits().stream()
207+
.map(KafkaPartitionSplit::splitId)
208+
.collect(Collectors.toSet()));
209+
List<TopicPartition> finishedPartitions =
210+
splitsRemoval.splits().stream()
211+
.map(KafkaPartitionSplit::getTopicPartition)
212+
.collect(Collectors.toList());
213+
finishedPartitions.forEach(kafkaSourceReaderMetrics::removeRecordsLagMetric);
214+
unassignPartitions(finishedPartitions);
215+
}
183216

217+
private void handleSplitsAddition(SplitsChange<KafkaPartitionSplit> splitsAddition) {
184218
// Assignment.
185219
List<TopicPartition> newPartitionAssignments = new ArrayList<>();
186220
// Starting offsets.
@@ -192,7 +226,7 @@ public void handleSplitsChanges(SplitsChange<KafkaPartitionSplit> splitsChange)
192226
Set<TopicPartition> partitionsStoppingAtCommitted = new HashSet<>();
193227

194228
// Parse the starting and stopping offsets.
195-
splitsChange
229+
splitsAddition
196230
.splits()
197231
.forEach(
198232
s -> {
@@ -223,7 +257,7 @@ public void handleSplitsChanges(SplitsChange<KafkaPartitionSplit> splitsChange)
223257
// After acquiring the starting and stopping offsets, remove the empty splits if necessary.
224258
removeEmptySplits();
225259

226-
maybeLogSplitChangesHandlingResult(splitsChange);
260+
maybeLogSplitChangesHandlingResult(splitsAddition);
227261
}
228262

229263
@Override

0 commit comments

Comments
 (0)