/
KafkaIOTest.java
759 lines (638 loc) · 27.1 KB
/
KafkaIOTest.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
/*
* 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.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import javax.annotation.Nullable;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
import org.apache.beam.sdk.coders.BigEndianLongCoder;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.NeedsRunner;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.Distinct;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Flatten;
import org.apache.beam.sdk.transforms.Max;
import org.apache.beam.sdk.transforms.Min;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.Values;
import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionList;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.hamcrest.collection.IsIterableContainingInAnyOrder;
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.ExpectedException;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/**
* Tests of {@link KafkaIO}.
*/
@RunWith(JUnit4.class)
public class KafkaIOTest {
/*
* The tests below borrow code and structure from CountingSourceTest. In addition verifies
* the reader interleaves the records from multiple partitions.
*
* Other tests to consider :
* - test KafkaRecordCoder
*/
@Rule
public ExpectedException thrown = ExpectedException.none();
// Update mock consumer with records distributed among the given topics, each with given number
// of partitions. Records are assigned in round-robin order among the partitions.
private static MockConsumer<byte[], byte[]> mkMockConsumer(
List<String> topics, int partitionsPerTopic, int numElements,
OffsetResetStrategy offsetResetStrategy) {
final List<TopicPartition> partitions = new ArrayList<>();
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> records = new HashMap<>();
Map<String, List<PartitionInfo>> partitionMap = new HashMap<>();
for (String topic : topics) {
List<PartitionInfo> partIds = new ArrayList<>(partitionsPerTopic);
for (int i = 0; i < partitionsPerTopic; i++) {
TopicPartition tp = new TopicPartition(topic, i);
partitions.add(tp);
partIds.add(new PartitionInfo(topic, i, null, null, null));
records.put(tp, new ArrayList<ConsumerRecord<byte[], byte[]>>());
}
partitionMap.put(topic, partIds);
}
int numPartitions = partitions.size();
long[] offsets = new long[numPartitions];
for (int i = 0; i < numElements; i++) {
int pIdx = i % numPartitions;
TopicPartition tp = partitions.get(pIdx);
records.get(tp).add(
new ConsumerRecord<>(
tp.topic(),
tp.partition(),
offsets[pIdx]++,
ByteBuffer.wrap(new byte[4]).putInt(i).array(), // key is 4 byte record id
ByteBuffer.wrap(new byte[8]).putLong(i).array())); // value is 8 byte record id
}
// This is updated when reader assigns partitions.
final AtomicReference<List<TopicPartition>> assignedPartitions =
new AtomicReference<>(Collections.<TopicPartition>emptyList());
final MockConsumer<byte[], byte[]> consumer =
new MockConsumer<byte[], byte[]>(offsetResetStrategy) {
// override assign() in order to set offset limits & to save assigned partitions.
public void assign(final List<TopicPartition> assigned) {
super.assign(assigned);
assignedPartitions.set(ImmutableList.copyOf(assigned));
for (TopicPartition tp : assigned) {
updateBeginningOffsets(ImmutableMap.of(tp, 0L));
updateEndOffsets(ImmutableMap.of(tp, (long) records.get(tp).size()));
}
}
};
for (String topic : topics) {
consumer.updatePartitions(topic, partitionMap.get(topic));
}
// MockConsumer does not maintain any relationship between partition seek position and the
// records added. e.g. if we add 10 records to a partition and then seek to end of the
// partition, MockConsumer is still going to return the 10 records in next poll. It is
// our responsibility to make sure currently enqueued records sync with partition offsets.
// The following task will be called inside each invocation to MockConsumer.poll().
// We enqueue only the records with the offset >= partition's current position.
Runnable recordEnquerTask = new Runnable() {
@Override
public void run() {
// add all the records with offset >= current partition position.
for (TopicPartition tp : assignedPartitions.get()) {
long curPos = consumer.position(tp);
for (ConsumerRecord<byte[], byte[]> r : records.get(tp)) {
if (r.offset() >= curPos) {
consumer.addRecord(r);
}
}
}
consumer.schedulePollTask(this);
}
};
consumer.schedulePollTask(recordEnquerTask);
return consumer;
}
private static class ConsumerFactoryFn
implements SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> {
private final List<String> topics;
private final int partitionsPerTopic;
private final int numElements;
private final OffsetResetStrategy offsetResetStrategy;
public ConsumerFactoryFn(List<String> topics,
int partitionsPerTopic,
int numElements,
OffsetResetStrategy offsetResetStrategy) {
this.topics = topics;
this.partitionsPerTopic = partitionsPerTopic;
this.numElements = numElements;
this.offsetResetStrategy = offsetResetStrategy;
}
public Consumer<byte[], byte[]> apply(Map<String, Object> config) {
return mkMockConsumer(topics, partitionsPerTopic, numElements, offsetResetStrategy);
}
}
/**
* Creates a consumer with two topics, with 5 partitions each.
* numElements are (round-robin) assigned all the 10 partitions.
*/
private static KafkaIO.TypedRead<Integer, Long> mkKafkaReadTransform(
int numElements,
@Nullable SerializableFunction<KV<Integer, Long>, Instant> timestampFn) {
List<String> topics = ImmutableList.of("topic_a", "topic_b");
KafkaIO.Read<Integer, Long> reader = KafkaIO.read()
.withBootstrapServers("none")
.withTopics(topics)
.withConsumerFactoryFn(new ConsumerFactoryFn(
topics, 10, numElements, OffsetResetStrategy.EARLIEST)) // 20 partitions
.withKeyCoder(BigEndianIntegerCoder.of())
.withValueCoder(BigEndianLongCoder.of())
.withMaxNumRecords(numElements);
if (timestampFn != null) {
return reader.withTimestampFn(timestampFn);
} else {
return reader;
}
}
private static class AssertMultipleOf implements SerializableFunction<Iterable<Long>, Void> {
private final int num;
public AssertMultipleOf(int num) {
this.num = num;
}
@Override
public Void apply(Iterable<Long> values) {
for (Long v : values) {
assertEquals(0, v % num);
}
return null;
}
}
public static void addCountingAsserts(PCollection<Long> input, long numElements) {
// Count == numElements
PAssert
.thatSingleton(input.apply("Count", Count.<Long>globally()))
.isEqualTo(numElements);
// Unique count == numElements
PAssert
.thatSingleton(input.apply(Distinct.<Long>create())
.apply("UniqueCount", Count.<Long>globally()))
.isEqualTo(numElements);
// Min == 0
PAssert
.thatSingleton(input.apply("Min", Min.<Long>globally()))
.isEqualTo(0L);
// Max == numElements-1
PAssert
.thatSingleton(input.apply("Max", Max.<Long>globally()))
.isEqualTo(numElements - 1);
}
@Test
@Category(NeedsRunner.class)
public void testUnboundedSource() {
Pipeline p = TestPipeline.create();
int numElements = 1000;
PCollection<Long> input = p
.apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn())
.withoutMetadata())
.apply(Values.<Long>create());
addCountingAsserts(input, numElements);
p.run();
}
@Test
@Category(NeedsRunner.class)
public void testUnboundedSourceWithExplicitPartitions() {
Pipeline p = TestPipeline.create();
int numElements = 1000;
List<String> topics = ImmutableList.of("test");
KafkaIO.TypedRead<byte[], Long> reader = KafkaIO.read()
.withBootstrapServers("none")
.withTopicPartitions(ImmutableList.of(new TopicPartition("test", 5)))
.withConsumerFactoryFn(new ConsumerFactoryFn(
topics, 10, numElements, OffsetResetStrategy.EARLIEST)) // 10 partitions
.withValueCoder(BigEndianLongCoder.of())
.withMaxNumRecords(numElements / 10);
PCollection<Long> input = p
.apply(reader.withoutMetadata())
.apply(Values.<Long>create());
// assert that every element is a multiple of 5.
PAssert
.that(input)
.satisfies(new AssertMultipleOf(5));
PAssert
.thatSingleton(input.apply(Count.<Long>globally()))
.isEqualTo(numElements / 10L);
p.run();
}
private static class ElementValueDiff extends DoFn<Long, Long> {
@ProcessElement
public void processElement(ProcessContext c) throws Exception {
c.output(c.element() - c.timestamp().getMillis());
}
}
@Test
@Category(NeedsRunner.class)
public void testUnboundedSourceTimestamps() {
Pipeline p = TestPipeline.create();
int numElements = 1000;
PCollection<Long> input = p
.apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn()).withoutMetadata())
.apply(Values.<Long>create());
addCountingAsserts(input, numElements);
PCollection<Long> diffs = input
.apply("TimestampDiff", ParDo.of(new ElementValueDiff()))
.apply("DistinctTimestamps", Distinct.<Long>create());
// This assert also confirms that diffs only has one unique value.
PAssert.thatSingleton(diffs).isEqualTo(0L);
p.run();
}
private static class RemoveKafkaMetadata<K, V> extends DoFn<KafkaRecord<K, V>, KV<K, V>> {
@ProcessElement
public void processElement(ProcessContext ctx) throws Exception {
ctx.output(ctx.element().getKV());
}
}
@Test
@Category(NeedsRunner.class)
public void testUnboundedSourceSplits() throws Exception {
Pipeline p = TestPipeline.create();
int numElements = 1000;
int numSplits = 10;
UnboundedSource<KafkaRecord<Integer, Long>, ?> initial =
mkKafkaReadTransform(numElements, null).makeSource();
List<? extends UnboundedSource<KafkaRecord<Integer, Long>, ?>> splits =
initial.generateInitialSplits(numSplits, p.getOptions());
assertEquals("Expected exact splitting", numSplits, splits.size());
long elementsPerSplit = numElements / numSplits;
assertEquals("Expected even splits", numElements, elementsPerSplit * numSplits);
PCollectionList<Long> pcollections = PCollectionList.empty(p);
for (int i = 0; i < splits.size(); ++i) {
pcollections = pcollections.and(
p.apply("split" + i, Read.from(splits.get(i)).withMaxNumRecords(elementsPerSplit))
.apply("Remove Metadata " + i, ParDo.of(new RemoveKafkaMetadata<Integer, Long>()))
.apply("collection " + i, Values.<Long>create()));
}
PCollection<Long> input = pcollections.apply(Flatten.<Long>pCollections());
addCountingAsserts(input, numElements);
p.run();
}
/**
* A timestamp function that uses the given value as the timestamp.
*/
private static class ValueAsTimestampFn
implements SerializableFunction<KV<Integer, Long>, Instant> {
@Override
public Instant apply(KV<Integer, Long> input) {
return new Instant(input.getValue());
}
}
// Kafka records are read in a separate thread inside the reader. As a result advance() might not
// read any records even from the mock consumer, especially for the first record.
// This is a helper method to loop until we read a record.
private static void advanceOnce(UnboundedReader<?> reader, boolean isStarted) throws IOException {
if (!isStarted && reader.start()) {
return;
}
while (!reader.advance()) {
// very rarely will there be more than one attempts.
// In case of a bug we might end up looping forever, and test will fail with a timeout.
// Avoid hard cpu spinning in case of a test failure.
try {
Thread.sleep(1);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
}
@Test
public void testUnboundedSourceCheckpointMark() throws Exception {
int numElements = 85; // 85 to make sure some partitions have more records than other.
// create a single split:
UnboundedSource<KafkaRecord<Integer, Long>, KafkaCheckpointMark> source =
mkKafkaReadTransform(numElements, new ValueAsTimestampFn())
.makeSource()
.generateInitialSplits(1, PipelineOptionsFactory.fromArgs(new String[0]).create())
.get(0);
UnboundedReader<KafkaRecord<Integer, Long>> reader = source.createReader(null, null);
final int numToSkip = 20; // one from each partition.
// advance numToSkip elements
for (int i = 0; i < numToSkip; ++i) {
advanceOnce(reader, i > 0);
}
// Confirm that we get the expected element in sequence before checkpointing.
assertEquals(numToSkip - 1, (long) reader.getCurrent().getKV().getValue());
assertEquals(numToSkip - 1, reader.getCurrentTimestamp().getMillis());
// Checkpoint and restart, and confirm that the source continues correctly.
KafkaCheckpointMark mark = CoderUtils.clone(
source.getCheckpointMarkCoder(), (KafkaCheckpointMark) reader.getCheckpointMark());
reader = source.createReader(null, mark);
// Confirm that we get the next elements in sequence.
// This also confirms that Reader interleaves records from each partitions by the reader.
for (int i = numToSkip; i < numElements; i++) {
advanceOnce(reader, i > numToSkip);
assertEquals(i, (long) reader.getCurrent().getKV().getValue());
assertEquals(i, reader.getCurrentTimestamp().getMillis());
}
}
@Test
public void testUnboundedSourceCheckpointMarkWithEmptyPartitions() throws Exception {
// Similar to testUnboundedSourceCheckpointMark(), but verifies that source resumes
// properly from empty partitions, without missing messages added since checkpoint.
// Initialize consumer with fewer elements than number of partitions so that some are empty.
int initialNumElements = 5;
UnboundedSource<KafkaRecord<Integer, Long>, KafkaCheckpointMark> source =
mkKafkaReadTransform(initialNumElements, new ValueAsTimestampFn())
.makeSource()
.generateInitialSplits(1, PipelineOptionsFactory.fromArgs(new String[0]).create())
.get(0);
UnboundedReader<KafkaRecord<Integer, Long>> reader = source.createReader(null, null);
for (int l = 0; l < initialNumElements; ++l) {
advanceOnce(reader, l > 0);
}
// Checkpoint and restart, and confirm that the source continues correctly.
KafkaCheckpointMark mark = CoderUtils.clone(
source.getCheckpointMarkCoder(), (KafkaCheckpointMark) reader.getCheckpointMark());
// Create another source with MockConsumer with OffsetResetStrategy.LATEST. This insures that
// the reader need to explicitly need to seek to first offset for partitions that were empty.
int numElements = 100; // all the 20 partitions will have elements
List<String> topics = ImmutableList.of("topic_a", "topic_b");
source = KafkaIO.read()
.withBootstrapServers("none")
.withTopics(topics)
.withConsumerFactoryFn(new ConsumerFactoryFn(
topics, 10, numElements, OffsetResetStrategy.LATEST))
.withKeyCoder(BigEndianIntegerCoder.of())
.withValueCoder(BigEndianLongCoder.of())
.withMaxNumRecords(numElements)
.withTimestampFn(new ValueAsTimestampFn())
.makeSource()
.generateInitialSplits(1, PipelineOptionsFactory.fromArgs(new String[0]).create())
.get(0);
reader = source.createReader(null, mark);
// Verify in any order. As the partitions are unevenly read, the returned records are not in a
// simple order. Note that testUnboundedSourceCheckpointMark() verifies round-robin oder.
List<Long> expected = new ArrayList<>();
List<Long> actual = new ArrayList<>();
for (long i = initialNumElements; i < numElements; i++) {
advanceOnce(reader, i > initialNumElements);
expected.add(i);
actual.add(reader.getCurrent().getKV().getValue());
}
assertThat(actual, IsIterableContainingInAnyOrder.containsInAnyOrder(expected.toArray()));
}
@Test
public void testSink() throws Exception {
// Simply read from kafka source and write to kafka sink. Then verify the records
// are correctly published to mock kafka producer.
int numElements = 1000;
synchronized (MOCK_PRODUCER_LOCK) {
MOCK_PRODUCER.clear();
ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread().start();
Pipeline pipeline = TestPipeline.create();
String topic = "test";
pipeline
.apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn())
.withoutMetadata())
.apply(KafkaIO.write()
.withBootstrapServers("none")
.withTopic(topic)
.withKeyCoder(BigEndianIntegerCoder.of())
.withValueCoder(BigEndianLongCoder.of())
.withProducerFactoryFn(new ProducerFactoryFn()));
pipeline.run();
completionThread.shutdown();
verifyProducerRecords(topic, numElements, false);
}
}
@Test
public void testValuesSink() throws Exception {
// similar to testSink(), but use values()' interface.
int numElements = 1000;
synchronized (MOCK_PRODUCER_LOCK) {
MOCK_PRODUCER.clear();
ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread().start();
Pipeline pipeline = TestPipeline.create();
String topic = "test";
pipeline
.apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn())
.withoutMetadata())
.apply(Values.<Long>create()) // there are no keys
.apply(KafkaIO.write()
.withBootstrapServers("none")
.withTopic(topic)
.withKeyCoder(BigEndianIntegerCoder.of())
.withValueCoder(BigEndianLongCoder.of())
.withProducerFactoryFn(new ProducerFactoryFn())
.values());
pipeline.run();
completionThread.shutdown();
verifyProducerRecords(topic, numElements, true);
}
}
@Test
public void testSinkWithSendErrors() throws Throwable {
// similar to testSink(), except that up to 10 of the send calls to producer will fail
// asynchronously.
// TODO: Ideally we want the pipeline to run to completion by retrying bundles that fail.
// We limit the number of errors injected to 10 below. This would reflect a real streaming
// pipeline. But I am sure how to achieve that. For now expect an exception:
thrown.expect(InjectedErrorException.class);
thrown.expectMessage("Injected Error #1");
int numElements = 1000;
synchronized (MOCK_PRODUCER_LOCK) {
MOCK_PRODUCER.clear();
Pipeline pipeline = TestPipeline.create();
String topic = "test";
ProducerSendCompletionThread completionThreadWithErrors =
new ProducerSendCompletionThread(10, 100).start();
pipeline
.apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn())
.withoutMetadata())
.apply(KafkaIO.write()
.withBootstrapServers("none")
.withTopic(topic)
.withKeyCoder(BigEndianIntegerCoder.of())
.withValueCoder(BigEndianLongCoder.of())
.withProducerFactoryFn(new ProducerFactoryFn()));
try {
pipeline.run();
} catch (PipelineExecutionException e) {
// throwing inner exception helps assert that first exception is thrown from the Sink
throw e.getCause().getCause();
} finally {
completionThreadWithErrors.shutdown();
}
}
}
private static void verifyProducerRecords(String topic, int numElements, boolean keyIsAbsent) {
// verify that appropriate messages are written to kafka
List<ProducerRecord<Integer, Long>> sent = MOCK_PRODUCER.history();
// sort by values
Collections.sort(sent, new Comparator<ProducerRecord<Integer, Long>>() {
@Override
public int compare(ProducerRecord<Integer, Long> o1, ProducerRecord<Integer, Long> o2) {
return Long.compare(o1.value(), o2.value());
}
});
for (int i = 0; i < numElements; i++) {
ProducerRecord<Integer, Long> record = sent.get(i);
assertEquals(topic, record.topic());
if (keyIsAbsent) {
assertNull(record.key());
} else {
assertEquals(i, record.key().intValue());
}
assertEquals(i, record.value().longValue());
}
}
/**
* Singleton MockProudcer. Using a singleton here since we need access to the object to fetch
* the actual records published to the producer. This prohibits running the tests using
* the producer in parallel, but there are only one or two tests.
*/
private static final MockProducer<Integer, Long> MOCK_PRODUCER =
new MockProducer<Integer, Long>(
false, // disable synchronous completion of send. see ProducerSendCompletionThread below.
new KafkaIO.CoderBasedKafkaSerializer<Integer>(),
new KafkaIO.CoderBasedKafkaSerializer<Long>()) {
// override flush() so that it does not complete all the waiting sends, giving a chance to
// ProducerCompletionThread to inject errors.
@Override
public void flush() {
while (completeNext()) {
// there are some uncompleted records. let the completion thread handle them.
try {
Thread.sleep(10);
} catch (InterruptedException e) {
}
}
}
};
// use a separate object serialize tests using MOCK_PRODUCER so that we don't interfere
// with Kafka MockProducer locking itself.
private static final Object MOCK_PRODUCER_LOCK = new Object();
private static class ProducerFactoryFn
implements SerializableFunction<Map<String, Object>, Producer<Integer, Long>> {
@Override
public Producer<Integer, Long> apply(Map<String, Object> config) {
return MOCK_PRODUCER;
}
}
private static class InjectedErrorException extends RuntimeException {
public InjectedErrorException(String message) {
super(message);
}
}
/**
* We start MockProducer with auto-completion disabled. That implies a record is not marked sent
* until #completeNext() is called on it. This class starts a thread to asynchronously 'complete'
* the the sends. During completion, we can also make those requests fail. This error injection
* is used in one of the tests.
*/
private static class ProducerSendCompletionThread {
private final int maxErrors;
private final int errorFrequency;
private final AtomicBoolean done = new AtomicBoolean(false);
private final ExecutorService injectorThread;
private int numCompletions = 0;
ProducerSendCompletionThread() {
// complete everything successfully
this(0, 0);
}
ProducerSendCompletionThread(final int maxErrors, final int errorFrequency) {
this.maxErrors = maxErrors;
this.errorFrequency = errorFrequency;
injectorThread = Executors.newSingleThreadExecutor();
}
ProducerSendCompletionThread start() {
injectorThread.submit(new Runnable() {
@Override
public void run() {
int errorsInjected = 0;
while (!done.get()) {
boolean successful;
if (errorsInjected < maxErrors && ((numCompletions + 1) % errorFrequency) == 0) {
successful = MOCK_PRODUCER.errorNext(
new InjectedErrorException("Injected Error #" + (errorsInjected + 1)));
if (successful) {
errorsInjected++;
}
} else {
successful = MOCK_PRODUCER.completeNext();
}
if (successful) {
numCompletions++;
} else {
// wait a bit since there are no unsent records
try {
Thread.sleep(1);
} catch (InterruptedException e) {
}
}
}
}
});
return this;
}
void shutdown() {
done.set(true);
injectorThread.shutdown();
try {
assertTrue(injectorThread.awaitTermination(10, TimeUnit.SECONDS));
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}
}
}