-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
KafkaIO.java
1695 lines (1491 loc) · 73.1 KB
/
KafkaIO.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
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.beam.sdk.io.kafka;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
import com.google.auto.service.AutoService;
import com.google.auto.value.AutoValue;
import io.confluent.kafka.serializers.KafkaAvroDeserializer;
import java.io.InputStream;
import java.io.OutputStream;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import javax.annotation.Nullable;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.coders.AtomicCoder;
import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderRegistry;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.coders.VarLongCoder;
import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
import org.apache.beam.sdk.io.Read.Unbounded;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.AppInfoParser;
import org.joda.time.Duration;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* An unbounded source and a sink for <a href="http://kafka.apache.org/">Kafka</a> topics.
*
* <h3>Reading from Kafka topics</h3>
*
* <p>KafkaIO source returns unbounded collection of Kafka records as {@code
* PCollection<KafkaRecord<K, V>>}. A {@link KafkaRecord} includes basic metadata like
* topic-partition and offset, along with key and value associated with a Kafka record.
*
* <p>Although most applications consume a single topic, the source can be configured to consume
* multiple topics or even a specific set of {@link TopicPartition}s.
*
* <p>To configure a Kafka source, you must specify at the minimum Kafka <tt>bootstrapServers</tt>,
* one or more topics to consume, and key and value deserializers. For example:
*
* <pre>{@code
* pipeline
* .apply(KafkaIO.<Long, String>read()
* .withBootstrapServers("broker_1:9092,broker_2:9092")
* .withTopic("my_topic") // use withTopics(List<String>) to read from multiple topics.
* .withKeyDeserializer(LongDeserializer.class)
* .withValueDeserializer(StringDeserializer.class)
*
* // Above four are required configuration. returns PCollection<KafkaRecord<Long, String>>
*
* // Rest of the settings are optional :
*
* // you can further customize KafkaConsumer used to read the records by adding more
* // settings for ConsumerConfig. e.g :
* .withConsumerConfigUpdates(ImmutableMap.of("group.id", "my_beam_app_1"))
*
* // set event times and watermark based on 'LogAppendTime'. To provide a custom
* // policy see withTimestampPolicyFactory(). withProcessingTime() is the default.
* // Use withCreateTime() with topics that have 'CreateTime' timestamps.
* .withLogAppendTime()
*
* // restrict reader to committed messages on Kafka (see method documentation).
* .withReadCommitted()
*
* // offset consumed by the pipeline can be committed back.
* .commitOffsetsInFinalize()
*
* // finally, if you don't need Kafka metadata, you can drop it.g
* .withoutMetadata() // PCollection<KV<Long, String>>
* )
* .apply(Values.<String>create()) // PCollection<String>
* ...
* }</pre>
*
* <p>Kafka provides deserializers for common types in {@link
* org.apache.kafka.common.serialization}. In addition to deserializers, Beam runners need {@link
* Coder} to materialize key and value objects if necessary. In most cases, you don't need to
* specify {@link Coder} for key and value in the resulting collection because the coders are
* inferred from deserializer types. However, in cases when coder inference fails, they can be
* specified explicitly along with deserializers using {@link
* Read#withKeyDeserializerAndCoder(Class, Coder)} and {@link
* Read#withValueDeserializerAndCoder(Class, Coder)}. Note that Kafka messages are interpreted using
* key and value <i>deserializers</i>.
*
* <h3>Partition Assignment and Checkpointing</h3>
*
* The Kafka partitions are evenly distributed among splits (workers).
*
* <p>Checkpointing is fully supported and each split can resume from previous checkpoint (to the
* extent supported by runner). See {@link KafkaUnboundedSource#split(int, PipelineOptions)} for
* more details on splits and checkpoint support.
*
* <p>When the pipeline starts for the first time, or without any checkpoint, the source starts
* consuming from the <em>latest</em> offsets. You can override this behavior to consume from the
* beginning by setting appropriate appropriate properties in {@link ConsumerConfig}, through {@link
* Read#withConsumerConfigUpdates(Map)}. You can also enable offset auto_commit in Kafka to resume
* from last committed.
*
* <p>In summary, KafkaIO.read follows below sequence to set initial offset:<br>
* 1. {@link KafkaCheckpointMark} provided by runner;<br>
* 2. Consumer offset stored in Kafka when {@code ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG = true};
* <br>
* 3. Start from <em>latest</em> offset by default;
*
* <p>Seek to initial offset is a blocking operation in Kafka API, which can block forever for
* certain versions of Kafka client library. This is resolved by <a
* href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-266%3A+Fix+consumer+indefinite+blocking+behavior">KIP-266</a>
* which provides `default.api.timeout.ms` consumer config setting to control such timeouts.
* KafkaIO.read implements timeout itself, to not to block forever in case older Kafka client is
* used. It does recognize `default.api.timeout.ms` setting and will honor the timeout value if it
* is passes in consumer config.
*
* <h3>Use Avro schema with Confluent Schema Registry</h3>
*
* <p>If you want to deserialize the keys and/or values based on a schema available in Confluent
* Schema Registry, KafkaIO can fetch this schema from a specified Schema Registry URL and use it
* for deserialization. A {@link Coder} will be inferred automatically based on the respective
* {@link Deserializer}.
*
* <p>For an Avro schema it will return a {@link PCollection} of {@link KafkaRecord}s where key
* and/or value will be typed as {@link org.apache.avro.generic.GenericRecord}. In this case, users
* don't need to specify key or/and value deserializers and coders since they will be set to {@link
* KafkaAvroDeserializer} and {@link AvroCoder} by default accordingly.
*
* <p>For example, below topic values are serialized with Avro schema stored in Schema Registry,
* keys are typed as {@link Long}:
*
* <pre>{@code
* PCollection<KafkaRecord<Long, GenericRecord>> input = pipeline
* .apply(KafkaIO.<Long, GenericRecord>read()
* .withBootstrapServers("broker_1:9092,broker_2:9092")
* .withTopic("my_topic")
* .withKeyDeserializer(LongDeserializer.class)
* // Use Confluent Schema Registry, specify schema registry URL and value subject
* .withValueDeserializer(
* ConfluentSchemaRegistryDeserializerProvider.of("http://localhost:8081", "my_topic-value"))
* ...
* }</pre>
*
* <h3>Writing to Kafka</h3>
*
* <p>KafkaIO sink supports writing key-value pairs to a Kafka topic. Users can also write just the
* values or native Kafka producer records using {@link
* org.apache.kafka.clients.producer.ProducerRecord}. To configure a Kafka sink, you must specify at
* the minimum Kafka <tt>bootstrapServers</tt>, the topic to write to, and key and value
* serializers. For example:
*
* <pre>{@code
* PCollection<KV<Long, String>> kvColl = ...;
* kvColl.apply(KafkaIO.<Long, String>write()
* .withBootstrapServers("broker_1:9092,broker_2:9092")
* .withTopic("results")
*
* .withKeySerializer(LongSerializer.class)
* .withValueSerializer(StringSerializer.class)
*
* // You can further customize KafkaProducer used to write the records by adding more
* // settings for ProducerConfig. e.g, to enable compression :
* .withProducerConfigUpdates(ImmutableMap.of("compression.type", "gzip"))
*
* // You set publish timestamp for the Kafka records.
* .withInputTimestamp() // element timestamp is used while publishing to Kafka
* // or you can also set a custom timestamp with a function.
* .withPublishTimestampFunction((elem, elemTs) -> ...)
*
* // Optionally enable exactly-once sink (on supported runners). See JavaDoc for withEOS().
* .withEOS(20, "eos-sink-group-id");
* );
* }</pre>
*
* <p>Often you might want to write just values without any keys to Kafka. Use {@code values()} to
* write records with default empty(null) key:
*
* <pre>{@code
* PCollection<String> strings = ...;
* strings.apply(KafkaIO.<Void, String>write()
* .withBootstrapServers("broker_1:9092,broker_2:9092")
* .withTopic("results")
* .withValueSerializer(StringSerializer.class) // just need serializer for value
* .values()
* );
* }</pre>
*
* <p>Also, if you want to write Kafka {@link ProducerRecord} then you should use {@link
* KafkaIO#writeRecords()}:
*
* <pre>{@code
* PCollection<ProducerRecord<Long, String>> records = ...;
* records.apply(KafkaIO.<Long, String>writeRecords()
* .withBootstrapServers("broker_1:9092,broker_2:9092")
* .withTopic("results")
* .withKeySerializer(LongSerializer.class)
* .withValueSerializer(StringSerializer.class)
* );
* }</pre>
*
* <h3>Advanced Kafka Configuration</h3>
*
* KafkaIO allows setting most of the properties in {@link ConsumerConfig} for source or in {@link
* ProducerConfig} for sink. E.g. if you would like to enable offset <em>auto commit</em> (for
* external monitoring or other purposes), you can set <tt>"group.id"</tt>,
* <tt>"enable.auto.commit"</tt>, etc.
*
* <h3>Event Timestamps and Watermark</h3>
*
* By default, record timestamp (event time) is set to processing time in KafkaIO reader and source
* watermark is current wall time. If a topic has Kafka server-side ingestion timestamp enabled
* ('LogAppendTime'), it can enabled with {@link Read#withLogAppendTime()}. A custom timestamp
* policy can be provided by implementing {@link TimestampPolicyFactory}. See {@link
* Read#withTimestampPolicyFactory(TimestampPolicyFactory)} for more information.
*
* <h3>Supported Kafka Client Versions</h3>
*
* KafkaIO relies on <i>kafka-clients</i> for all its interactions with the Kafka cluster.
* <i>kafka-clients</i> versions 0.10.1 and newer are supported at runtime. The older versions 0.9.x
* - 0.10.0.0 are also supported, but are deprecated and likely be removed in near future. Please
* ensure that the version included with the application is compatible with the version of your
* Kafka cluster. Kafka client usually fails to initialize with a clear error message in case of
* incompatibility.
*/
@Experimental(Kind.SOURCE_SINK)
public class KafkaIO {
/**
* A specific instance of uninitialized {@link #read()} where key and values are bytes. See
* #read().
*/
public static Read<byte[], byte[]> readBytes() {
return KafkaIO.<byte[], byte[]>read()
.withKeyDeserializer(ByteArrayDeserializer.class)
.withValueDeserializer(ByteArrayDeserializer.class);
}
/**
* Creates an uninitialized {@link Read} {@link PTransform}. Before use, basic Kafka configuration
* should set with {@link Read#withBootstrapServers(String)} and {@link Read#withTopics(List)}.
* Other optional settings include key and value {@link Deserializer}s, custom timestamp and
* watermark functions.
*/
public static <K, V> Read<K, V> read() {
return new AutoValue_KafkaIO_Read.Builder<K, V>()
.setTopics(new ArrayList<>())
.setTopicPartitions(new ArrayList<>())
.setConsumerFactoryFn(Read.KAFKA_CONSUMER_FACTORY_FN)
.setConsumerConfig(Read.DEFAULT_CONSUMER_PROPERTIES)
.setMaxNumRecords(Long.MAX_VALUE)
.setCommitOffsetsInFinalizeEnabled(false)
.setTimestampPolicyFactory(TimestampPolicyFactory.withProcessingTime())
.build();
}
/**
* Creates an uninitialized {@link Write} {@link PTransform}. Before use, Kafka configuration
* should be set with {@link Write#withBootstrapServers(String)} and {@link Write#withTopic} along
* with {@link Deserializer}s for (optional) key and values.
*/
public static <K, V> Write<K, V> write() {
return new AutoValue_KafkaIO_Write.Builder<K, V>()
.setWriteRecordsTransform(
new AutoValue_KafkaIO_WriteRecords.Builder<K, V>()
.setProducerConfig(WriteRecords.DEFAULT_PRODUCER_PROPERTIES)
.setEOS(false)
.setNumShards(0)
.setConsumerFactoryFn(Read.KAFKA_CONSUMER_FACTORY_FN)
.build())
.build();
}
/**
* Creates an uninitialized {@link WriteRecords} {@link PTransform}. Before use, Kafka
* configuration should be set with {@link WriteRecords#withBootstrapServers(String)} and {@link
* WriteRecords#withTopic} along with {@link Deserializer}s for (optional) key and values.
*/
public static <K, V> WriteRecords<K, V> writeRecords() {
return new AutoValue_KafkaIO_WriteRecords.Builder<K, V>()
.setProducerConfig(WriteRecords.DEFAULT_PRODUCER_PROPERTIES)
.setEOS(false)
.setNumShards(0)
.setConsumerFactoryFn(Read.KAFKA_CONSUMER_FACTORY_FN)
.build();
}
///////////////////////// Read Support \\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\
/**
* A {@link PTransform} to read from Kafka topics. See {@link KafkaIO} for more information on
* usage and configuration.
*/
@AutoValue
public abstract static class Read<K, V>
extends PTransform<PBegin, PCollection<KafkaRecord<K, V>>> {
abstract Map<String, Object> getConsumerConfig();
abstract List<String> getTopics();
abstract List<TopicPartition> getTopicPartitions();
@Nullable
abstract Coder<K> getKeyCoder();
@Nullable
abstract Coder<V> getValueCoder();
abstract SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
getConsumerFactoryFn();
@Nullable
abstract SerializableFunction<KafkaRecord<K, V>, Instant> getWatermarkFn();
abstract long getMaxNumRecords();
@Nullable
abstract Duration getMaxReadTime();
@Nullable
abstract Instant getStartReadTime();
abstract boolean isCommitOffsetsInFinalizeEnabled();
abstract TimestampPolicyFactory<K, V> getTimestampPolicyFactory();
@Nullable
abstract Map<String, Object> getOffsetConsumerConfig();
@Nullable
abstract DeserializerProvider getKeyDeserializerProvider();
@Nullable
abstract DeserializerProvider getValueDeserializerProvider();
abstract Builder<K, V> toBuilder();
@Experimental(Kind.PORTABILITY)
@AutoValue.Builder
abstract static class Builder<K, V>
implements ExternalTransformBuilder<External.Configuration, PBegin, PCollection<KV<K, V>>> {
abstract Builder<K, V> setConsumerConfig(Map<String, Object> config);
abstract Builder<K, V> setTopics(List<String> topics);
abstract Builder<K, V> setTopicPartitions(List<TopicPartition> topicPartitions);
abstract Builder<K, V> setKeyCoder(Coder<K> keyCoder);
abstract Builder<K, V> setValueCoder(Coder<V> valueCoder);
abstract Builder<K, V> setConsumerFactoryFn(
SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> consumerFactoryFn);
abstract Builder<K, V> setWatermarkFn(SerializableFunction<KafkaRecord<K, V>, Instant> fn);
abstract Builder<K, V> setMaxNumRecords(long maxNumRecords);
abstract Builder<K, V> setMaxReadTime(Duration maxReadTime);
abstract Builder<K, V> setStartReadTime(Instant startReadTime);
abstract Builder<K, V> setCommitOffsetsInFinalizeEnabled(boolean commitOffsetInFinalize);
abstract Builder<K, V> setTimestampPolicyFactory(
TimestampPolicyFactory<K, V> timestampPolicyFactory);
abstract Builder<K, V> setOffsetConsumerConfig(Map<String, Object> offsetConsumerConfig);
abstract Builder<K, V> setKeyDeserializerProvider(DeserializerProvider deserializerProvider);
abstract Builder<K, V> setValueDeserializerProvider(
DeserializerProvider deserializerProvider);
abstract Read<K, V> build();
@Override
public PTransform<PBegin, PCollection<KV<K, V>>> buildExternal(
External.Configuration config) {
ImmutableList.Builder<String> listBuilder = ImmutableList.builder();
for (String topic : config.topics) {
listBuilder.add(topic);
}
setTopics(listBuilder.build());
Class keyDeserializer = resolveClass(config.keyDeserializer);
setKeyDeserializerProvider(LocalDeserializerProvider.of(keyDeserializer));
setKeyCoder(resolveCoder(keyDeserializer));
Class valueDeserializer = resolveClass(config.valueDeserializer);
setValueDeserializerProvider(LocalDeserializerProvider.of(valueDeserializer));
setValueCoder(resolveCoder(valueDeserializer));
Map<String, Object> consumerConfig = new HashMap<>();
for (KV<String, String> kv : config.consumerConfig) {
consumerConfig.put(kv.getKey(), kv.getValue());
}
// Key and Value Deserializers always have to be in the config.
consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getName());
consumerConfig.put(
ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getName());
setConsumerConfig(consumerConfig);
// Set required defaults
setTopicPartitions(Collections.emptyList());
setConsumerFactoryFn(Read.KAFKA_CONSUMER_FACTORY_FN);
setMaxNumRecords(Long.MAX_VALUE);
setCommitOffsetsInFinalizeEnabled(false);
setTimestampPolicyFactory(TimestampPolicyFactory.withProcessingTime());
// We do not include Metadata until we can encode KafkaRecords cross-language
return build().withoutMetadata();
}
private static Coder resolveCoder(Class deserializer) {
for (Method method : deserializer.getDeclaredMethods()) {
if (method.getName().equals("deserialize")) {
Class<?> returnType = method.getReturnType();
if (returnType.equals(Object.class)) {
continue;
}
if (returnType.equals(byte[].class)) {
return ByteArrayCoder.of();
} else if (returnType.equals(Integer.class)) {
return VarIntCoder.of();
} else if (returnType.equals(Long.class)) {
return VarLongCoder.of();
} else {
throw new RuntimeException("Couldn't infer Coder from " + deserializer);
}
}
}
throw new RuntimeException("Couldn't resolve coder for Deserializer: " + deserializer);
}
}
/**
* Exposes {@link KafkaIO.TypedWithoutMetadata} as an external transform for cross-language
* usage.
*/
@Experimental(Kind.PORTABILITY)
@AutoService(ExternalTransformRegistrar.class)
public static class External implements ExternalTransformRegistrar {
public static final String URN = "beam:external:java:kafka:read:v1";
@Override
public Map<String, Class<? extends ExternalTransformBuilder>> knownBuilders() {
return ImmutableMap.of(URN, AutoValue_KafkaIO_Read.Builder.class);
}
/** Parameters class to expose the Read transform to an external SDK. */
public static class Configuration {
// All byte arrays are UTF-8 encoded strings
private Iterable<KV<String, String>> consumerConfig;
private Iterable<String> topics;
private String keyDeserializer;
private String valueDeserializer;
public void setConsumerConfig(Iterable<KV<String, String>> consumerConfig) {
this.consumerConfig = consumerConfig;
}
public void setTopics(Iterable<String> topics) {
this.topics = topics;
}
public void setKeyDeserializer(String keyDeserializer) {
this.keyDeserializer = keyDeserializer;
}
public void setValueDeserializer(String valueDeserializer) {
this.valueDeserializer = valueDeserializer;
}
}
}
/** Sets the bootstrap servers for the Kafka consumer. */
public Read<K, V> withBootstrapServers(String bootstrapServers) {
return withConsumerConfigUpdates(
ImmutableMap.of(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers));
}
/**
* Sets the topic to read from.
*
* <p>See {@link KafkaUnboundedSource#split(int, PipelineOptions)} for description of how the
* partitions are distributed among the splits.
*/
public Read<K, V> withTopic(String topic) {
return withTopics(ImmutableList.of(topic));
}
/**
* Sets a list of topics to read from. All the partitions from each of the topics are read.
*
* <p>See {@link KafkaUnboundedSource#split(int, PipelineOptions)} for description of how the
* partitions are distributed among the splits.
*/
public Read<K, V> withTopics(List<String> topics) {
checkState(
getTopicPartitions().isEmpty(), "Only topics or topicPartitions can be set, not both");
return toBuilder().setTopics(ImmutableList.copyOf(topics)).build();
}
/**
* Sets a list of partitions to read from. This allows reading only a subset of partitions for
* one or more topics when (if ever) needed.
*
* <p>See {@link KafkaUnboundedSource#split(int, PipelineOptions)} for description of how the
* partitions are distributed among the splits.
*/
public Read<K, V> withTopicPartitions(List<TopicPartition> topicPartitions) {
checkState(getTopics().isEmpty(), "Only topics or topicPartitions can be set, not both");
return toBuilder().setTopicPartitions(ImmutableList.copyOf(topicPartitions)).build();
}
/**
* Sets a Kafka {@link Deserializer} to interpret key bytes read from Kafka.
*
* <p>In addition, Beam also needs a {@link Coder} to serialize and deserialize key objects at
* runtime. KafkaIO tries to infer a coder for the key based on the {@link Deserializer} class,
* however in case that fails, you can use {@link #withKeyDeserializerAndCoder(Class, Coder)} to
* provide the key coder explicitly.
*/
public Read<K, V> withKeyDeserializer(Class<? extends Deserializer<K>> keyDeserializer) {
return withKeyDeserializer(LocalDeserializerProvider.of(keyDeserializer));
}
/**
* Sets a Kafka {@link Deserializer} for interpreting key bytes read from Kafka along with a
* {@link Coder} for helping the Beam runner materialize key objects at runtime if necessary.
*
* <p>Use this method only if your pipeline doesn't work with plain {@link
* #withKeyDeserializer(Class)}.
*/
public Read<K, V> withKeyDeserializerAndCoder(
Class<? extends Deserializer<K>> keyDeserializer, Coder<K> keyCoder) {
return withKeyDeserializer(keyDeserializer).toBuilder().setKeyCoder(keyCoder).build();
}
public Read<K, V> withKeyDeserializer(DeserializerProvider<K> deserializerProvider) {
return toBuilder().setKeyDeserializerProvider(deserializerProvider).build();
}
/**
* Sets a Kafka {@link Deserializer} to interpret value bytes read from Kafka.
*
* <p>In addition, Beam also needs a {@link Coder} to serialize and deserialize value objects at
* runtime. KafkaIO tries to infer a coder for the value based on the {@link Deserializer}
* class, however in case that fails, you can use {@link #withValueDeserializerAndCoder(Class,
* Coder)} to provide the value coder explicitly.
*/
public Read<K, V> withValueDeserializer(Class<? extends Deserializer<V>> valueDeserializer) {
return withValueDeserializer(LocalDeserializerProvider.of(valueDeserializer));
}
/**
* Sets a Kafka {@link Deserializer} for interpreting value bytes read from Kafka along with a
* {@link Coder} for helping the Beam runner materialize value objects at runtime if necessary.
*
* <p>Use this method only if your pipeline doesn't work with plain {@link
* #withValueDeserializer(Class)}.
*/
public Read<K, V> withValueDeserializerAndCoder(
Class<? extends Deserializer<V>> valueDeserializer, Coder<V> valueCoder) {
return withValueDeserializer(valueDeserializer).toBuilder().setValueCoder(valueCoder).build();
}
public Read<K, V> withValueDeserializer(DeserializerProvider<V> deserializerProvider) {
return toBuilder().setValueDeserializerProvider(deserializerProvider).build();
}
/**
* A factory to create Kafka {@link Consumer} from consumer configuration. This is useful for
* supporting another version of Kafka consumer. Default is {@link KafkaConsumer}.
*/
public Read<K, V> withConsumerFactoryFn(
SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> consumerFactoryFn) {
return toBuilder().setConsumerFactoryFn(consumerFactoryFn).build();
}
/**
* Update consumer configuration with new properties.
*
* @deprecated as of version 2.13. Use {@link #withConsumerConfigUpdates(Map)} instead
*/
@Deprecated
public Read<K, V> updateConsumerProperties(Map<String, Object> configUpdates) {
Map<String, Object> config =
updateKafkaProperties(getConsumerConfig(), IGNORED_CONSUMER_PROPERTIES, configUpdates);
return toBuilder().setConsumerConfig(config).build();
}
/**
* Similar to {@link org.apache.beam.sdk.io.Read.Unbounded#withMaxNumRecords(long)}. Mainly used
* for tests and demo applications.
*/
public Read<K, V> withMaxNumRecords(long maxNumRecords) {
return toBuilder().setMaxNumRecords(maxNumRecords).build();
}
/**
* Use timestamp to set up start offset. It is only supported by Kafka Client 0.10.1.0 onwards
* and the message format version after 0.10.0.
*
* <p>Note that this take priority over start offset configuration {@code
* ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} and any auto committed offsets.
*
* <p>This results in hard failures in either of the following two cases : 1. If one of more
* partitions do not contain any messages with timestamp larger than or equal to desired
* timestamp. 2. If the message format version in a partition is before 0.10.0, i.e. the
* messages do not have timestamps.
*/
public Read<K, V> withStartReadTime(Instant startReadTime) {
return toBuilder().setStartReadTime(startReadTime).build();
}
/**
* Similar to {@link org.apache.beam.sdk.io.Read.Unbounded#withMaxReadTime(Duration)}. Mainly
* used for tests and demo applications.
*/
public Read<K, V> withMaxReadTime(Duration maxReadTime) {
return toBuilder().setMaxReadTime(maxReadTime).build();
}
/**
* Sets {@link TimestampPolicy} to {@link TimestampPolicyFactory.LogAppendTimePolicy}. The
* policy assigns Kafka's log append time (server side ingestion time) to each record. The
* watermark for each Kafka partition is the timestamp of the last record read. If a partition
* is idle, the watermark advances to couple of seconds behind wall time. Every record consumed
* from Kafka is expected to have its timestamp type set to 'LOG_APPEND_TIME'.
*
* <p>In Kafka, log append time needs to be enabled for each topic, and all the subsequent
* records wil have their timestamp set to log append time. If a record does not have its
* timestamp type set to 'LOG_APPEND_TIME' for any reason, it's timestamp is set to previous
* record timestamp or latest watermark, whichever is larger.
*
* <p>The watermark for the entire source is the oldest of each partition's watermark. If one of
* the readers falls behind possibly due to uneven distribution of records among Kafka
* partitions, it ends up holding the watermark for the entire source.
*/
public Read<K, V> withLogAppendTime() {
return withTimestampPolicyFactory(TimestampPolicyFactory.withLogAppendTime());
}
/**
* Sets {@link TimestampPolicy} to {@link TimestampPolicyFactory.ProcessingTimePolicy}. This is
* the default timestamp policy. It assigns processing time to each record. Specifically, this
* is the timestamp when the record becomes 'current' in the reader. The watermark aways
* advances to current time. If server side time (log append time) is enabled in Kafka, {@link
* #withLogAppendTime()} is recommended over this.
*/
public Read<K, V> withProcessingTime() {
return withTimestampPolicyFactory(TimestampPolicyFactory.withProcessingTime());
}
/**
* Sets the timestamps policy based on {@link KafkaTimestampType#CREATE_TIME} timestamp of the
* records. It is an error if a record's timestamp type is not {@link
* KafkaTimestampType#CREATE_TIME}. The timestamps within a partition are expected to be roughly
* monotonically increasing with a cap on out of order delays (e.g. 'max delay' of 1 minute).
* The watermark at any time is '({@code Min(now(), Max(event timestamp so far)) - max delay})'.
* However, watermark is never set in future and capped to 'now - max delay'. In addition,
* watermark advanced to 'now - max delay' when a partition is idle.
*
* @param maxDelay For any record in the Kafka partition, the timestamp of any subsequent record
* is expected to be after {@code current record timestamp - maxDelay}.
*/
public Read<K, V> withCreateTime(Duration maxDelay) {
return withTimestampPolicyFactory(TimestampPolicyFactory.withCreateTime(maxDelay));
}
/**
* Provide custom {@link TimestampPolicyFactory} to set event times and watermark for each
* partition. {@link TimestampPolicyFactory#createTimestampPolicy(TopicPartition, Optional)} is
* invoked for each partition when the reader starts.
*
* @see #withLogAppendTime()
* @see #withCreateTime(Duration)
* @see #withProcessingTime()
*/
public Read<K, V> withTimestampPolicyFactory(
TimestampPolicyFactory<K, V> timestampPolicyFactory) {
return toBuilder().setTimestampPolicyFactory(timestampPolicyFactory).build();
}
/**
* A function to assign a timestamp to a record. Default is processing timestamp.
*
* @deprecated as of version 2.4. Use {@link
* #withTimestampPolicyFactory(TimestampPolicyFactory)} instead.
*/
@Deprecated
public Read<K, V> withTimestampFn2(
SerializableFunction<KafkaRecord<K, V>, Instant> timestampFn) {
checkArgument(timestampFn != null, "timestampFn can not be null");
return toBuilder()
.setTimestampPolicyFactory(TimestampPolicyFactory.withTimestampFn(timestampFn))
.build();
}
/**
* A function to calculate watermark after a record. Default is last record timestamp.
*
* @see #withTimestampFn(SerializableFunction)
* @deprecated as of version 2.4. Use {@link
* #withTimestampPolicyFactory(TimestampPolicyFactory)} instead.
*/
@Deprecated
public Read<K, V> withWatermarkFn2(
SerializableFunction<KafkaRecord<K, V>, Instant> watermarkFn) {
checkArgument(watermarkFn != null, "watermarkFn can not be null");
return toBuilder().setWatermarkFn(watermarkFn).build();
}
/**
* A function to assign a timestamp to a record. Default is processing timestamp.
*
* @deprecated as of version 2.4. Use {@link
* #withTimestampPolicyFactory(TimestampPolicyFactory)} instead.
*/
@Deprecated
public Read<K, V> withTimestampFn(SerializableFunction<KV<K, V>, Instant> timestampFn) {
checkArgument(timestampFn != null, "timestampFn can not be null");
return withTimestampFn2(unwrapKafkaAndThen(timestampFn));
}
/**
* A function to calculate watermark after a record. Default is last record timestamp.
*
* @see #withTimestampFn(SerializableFunction)
* @deprecated as of version 2.4. Use {@link
* #withTimestampPolicyFactory(TimestampPolicyFactory)} instead.
*/
@Deprecated
public Read<K, V> withWatermarkFn(SerializableFunction<KV<K, V>, Instant> watermarkFn) {
checkArgument(watermarkFn != null, "watermarkFn can not be null");
return withWatermarkFn2(unwrapKafkaAndThen(watermarkFn));
}
/**
* Sets "isolation_level" to "read_committed" in Kafka consumer configuration. This is ensures
* that the consumer does not read uncommitted messages. Kafka version 0.11 introduced
* transactional writes. Applications requiring end-to-end exactly-once semantics should only
* read committed messages. See JavaDoc for {@link KafkaConsumer} for more description.
*/
public Read<K, V> withReadCommitted() {
return withConsumerConfigUpdates(ImmutableMap.of("isolation.level", "read_committed"));
}
/**
* Finalized offsets are committed to Kafka. See {@link CheckpointMark#finalizeCheckpoint()}. It
* helps with minimizing gaps or duplicate processing of records while restarting a pipeline
* from scratch. But it does not provide hard processing guarantees. There could be a short
* delay to commit after {@link CheckpointMark#finalizeCheckpoint()} is invoked, as reader might
* be blocked on reading from Kafka. Note that it is independent of 'AUTO_COMMIT' Kafka consumer
* configuration. Usually either this or AUTO_COMMIT in Kafka consumer is enabled, but not both.
*/
public Read<K, V> commitOffsetsInFinalize() {
return toBuilder().setCommitOffsetsInFinalizeEnabled(true).build();
}
/**
* Set additional configuration for the backend offset consumer. It may be required for a
* secured Kafka cluster, especially when you see similar WARN log message 'exception while
* fetching latest offset for partition {}. will be retried'.
*
* <p>In {@link KafkaIO#read()}, there're two consumers running in the backend actually:<br>
* 1. the main consumer, which reads data from kafka;<br>
* 2. the secondary offset consumer, which is used to estimate backlog, by fetching latest
* offset;<br>
*
* <p>By default, offset consumer inherits the configuration from main consumer, with an
* auto-generated {@link ConsumerConfig#GROUP_ID_CONFIG}. This may not work in a secured Kafka
* which requires more configurations.
*/
public Read<K, V> withOffsetConsumerConfigOverrides(Map<String, Object> offsetConsumerConfig) {
return toBuilder().setOffsetConsumerConfig(offsetConsumerConfig).build();
}
/**
* Update configuration for the backend main consumer. Note that the default consumer properties
* will not be completely overridden. This method only updates the value which has the same key.
*
* <p>In {@link KafkaIO#read()}, there're two consumers running in the backend actually:<br>
* 1. the main consumer, which reads data from kafka;<br>
* 2. the secondary offset consumer, which is used to estimate backlog, by fetching latest
* offset;<br>
*
* <p>By default, main consumer uses the configuration from {@link
* #DEFAULT_CONSUMER_PROPERTIES}.
*/
public Read<K, V> withConsumerConfigUpdates(Map<String, Object> configUpdates) {
Map<String, Object> config =
updateKafkaProperties(getConsumerConfig(), IGNORED_CONSUMER_PROPERTIES, configUpdates);
return toBuilder().setConsumerConfig(config).build();
}
/** Returns a {@link PTransform} for PCollection of {@link KV}, dropping Kafka metatdata. */
public PTransform<PBegin, PCollection<KV<K, V>>> withoutMetadata() {
return new TypedWithoutMetadata<>(this);
}
@Override
public PCollection<KafkaRecord<K, V>> expand(PBegin input) {
checkArgument(
getConsumerConfig().get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG) != null,
"withBootstrapServers() is required");
checkArgument(
getTopics().size() > 0 || getTopicPartitions().size() > 0,
"Either withTopic(), withTopics() or withTopicPartitions() is required");
checkArgument(getKeyDeserializerProvider() != null, "withKeyDeserializer() is required");
checkArgument(getValueDeserializerProvider() != null, "withValueDeserializer() is required");
ConsumerSpEL consumerSpEL = new ConsumerSpEL();
if (!consumerSpEL.hasOffsetsForTimes()) {
LOG.warn(
"Kafka client version {} is too old. Versions before 0.10.1.0 are deprecated and "
+ "may not be supported in next release of Apache Beam. "
+ "Please upgrade your Kafka client version.",
AppInfoParser.getVersion());
}
if (getStartReadTime() != null) {
checkArgument(
consumerSpEL.hasOffsetsForTimes(),
"Consumer.offsetsForTimes is only supported by Kafka Client 0.10.1.0 onwards, "
+ "current version of Kafka Client is "
+ AppInfoParser.getVersion()
+ ". If you are building with maven, set \"kafka.clients.version\" "
+ "maven property to 0.10.1.0 or newer.");
}
if (isCommitOffsetsInFinalizeEnabled()) {
checkArgument(
getConsumerConfig().get(ConsumerConfig.GROUP_ID_CONFIG) != null,
"commitOffsetsInFinalize() is enabled, but group.id in Kafka consumer config "
+ "is not set. Offset management requires group.id.");
if (Boolean.TRUE.equals(
getConsumerConfig().get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG))) {
LOG.warn(
"'{}' in consumer config is enabled even though commitOffsetsInFinalize() "
+ "is set. You need only one of them.",
ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
}
}
// Infer key/value coders if not specified explicitly
CoderRegistry coderRegistry = input.getPipeline().getCoderRegistry();
Coder<K> keyCoder = getKeyCoder(coderRegistry);
Coder<V> valueCoder = getValueCoder(coderRegistry);
// Handles unbounded source to bounded conversion if maxNumRecords or maxReadTime is set.
Unbounded<KafkaRecord<K, V>> unbounded =
org.apache.beam.sdk.io.Read.from(
toBuilder().setKeyCoder(keyCoder).setValueCoder(valueCoder).build().makeSource());
PTransform<PBegin, PCollection<KafkaRecord<K, V>>> transform = unbounded;
if (getMaxNumRecords() < Long.MAX_VALUE || getMaxReadTime() != null) {
transform =
unbounded.withMaxReadTime(getMaxReadTime()).withMaxNumRecords(getMaxNumRecords());
}
return input.getPipeline().apply(transform);
}
private Coder<K> getKeyCoder(CoderRegistry coderRegistry) {
return (getKeyCoder() != null)
? getKeyCoder()
: getKeyDeserializerProvider().getCoder(coderRegistry);
}
private Coder<V> getValueCoder(CoderRegistry coderRegistry) {
return (getValueCoder() != null)
? getValueCoder()
: getValueDeserializerProvider().getCoder(coderRegistry);
}
/**
* Creates an {@link UnboundedSource UnboundedSource<KafkaRecord<K, V>, ?>} with the
* configuration in {@link Read}. Primary use case is unit tests, should not be used in an
* application.
*/
@VisibleForTesting
UnboundedSource<KafkaRecord<K, V>, KafkaCheckpointMark> makeSource() {
return new KafkaUnboundedSource<>(this, -1);
}
// utility method to convert KafkaRecord<K, V> to user KV<K, V> before applying user functions
private static <KeyT, ValueT, OutT>
SerializableFunction<KafkaRecord<KeyT, ValueT>, OutT> unwrapKafkaAndThen(
final SerializableFunction<KV<KeyT, ValueT>, OutT> fn) {
return record -> fn.apply(record.getKV());
}
///////////////////////////////////////////////////////////////////////////////////////
/** A set of properties that are not required or don't make sense for our consumer. */
private static final Map<String, String> IGNORED_CONSUMER_PROPERTIES =
ImmutableMap.of(
ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDeserializer instead",
ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueDeserializer instead"
// "group.id", "enable.auto.commit", "auto.commit.interval.ms" :
// lets allow these, applications can have better resume point for restarts.
);
// set config defaults
private static final Map<String, Object> DEFAULT_CONSUMER_PROPERTIES =
ImmutableMap.of(
ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
ByteArrayDeserializer.class.getName(),
ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
ByteArrayDeserializer.class.getName(),
// Use large receive buffer. Once KAFKA-3135 is fixed, this _may_ not be required.
// with default value of of 32K, It takes multiple seconds between successful polls.
// All the consumer work is done inside poll(), with smaller send buffer size, it
// takes many polls before a 1MB chunk from the server is fully read. In my testing
// about half of the time select() inside kafka consumer waited for 20-30ms, though
// the server had lots of data in tcp send buffers on its side. Compared to default,
// this setting increased throughput by many fold (3-4x).
ConsumerConfig.RECEIVE_BUFFER_CONFIG,
512 * 1024,
// default to latest offset when we are not resuming.
ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
"latest",
// disable auto commit of offsets. we don't require group_id. could be enabled by user.
ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
false);
// default Kafka 0.9 Consumer supplier.
private static final SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
KAFKA_CONSUMER_FACTORY_FN = KafkaConsumer::new;
@SuppressWarnings("unchecked")
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
List<String> topics = getTopics();
List<TopicPartition> topicPartitions = getTopicPartitions();
if (topics.size() > 0) {
builder.add(DisplayData.item("topics", Joiner.on(",").join(topics)).withLabel("Topic/s"));
} else if (topicPartitions.size() > 0) {