Skip to content

Commit 3572fc8

Browse files
authored
Merge pull request #966: [direct-io-kafka] idle watermark robustness
2 parents 50c5c23 + 9373642 commit 3572fc8

File tree

3 files changed

+43
-37
lines changed

3 files changed

+43
-37
lines changed

direct/io-kafka/src/main/java/cz/o2/proxima/direct/io/kafka/KafkaLogReader.java

Lines changed: 27 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -402,6 +402,7 @@ private void submitConsumerWithObserver(
402402
new KafkaThroughputLimiter(maxBytesPerSec);
403403
final Map<TopicPartition, Long> polledOffsets = new HashMap<>();
404404
final Map<TopicPartition, Integer> emptyPolls = new HashMap<>();
405+
final Map<TopicPartition, Long> endOffsets = new HashMap<>();
405406

406407
handle.set(
407408
createObserveHandle(shutdown, seekOffsets, consumer, readyLatch, completedLatch));
@@ -414,6 +415,7 @@ private void submitConsumerWithObserver(
414415
topicPartitionToId,
415416
emptyPolls,
416417
polledOffsets,
418+
endOffsets,
417419
watermarkEstimator);
418420

419421
try (KafkaConsumer<Object, Object> kafka =
@@ -423,12 +425,9 @@ private void submitConsumerWithObserver(
423425

424426
// we need to poll first to initialize kafka assignments and rebalance listener
425427
ConsumerRecords<Object, Object> poll;
426-
Map<TopicPartition, Long> endOffsets;
427428

428429
do {
429430
poll = kafka.poll(pollDuration);
430-
endOffsets = stopAtCurrent ? findNonEmptyEndOffsets(kafka) : null;
431-
432431
if (log.isDebugEnabled()) {
433432
log.debug(
434433
"End offsets of current assignment {}: {}", kafka.assignment(), endOffsets);
@@ -498,7 +497,12 @@ private void submitConsumerWithObserver(
498497
terminateIfConsumed(stopAtCurrent, kafka, endOffsets, polledOffsets, completed);
499498

500499
progressWatermarkOnEmptyPartitions(
501-
consumer, emptyPolls, topicPartitionToId, watermarkEstimator);
500+
consumer,
501+
emptyPolls,
502+
endOffsets,
503+
polledOffsets,
504+
topicPartitionToId,
505+
watermarkEstimator);
502506
throughputLimiter.sleepToLimitThroughput(bytesPolled, pollTimeMs);
503507
long startTime = System.currentTimeMillis();
504508
poll = kafka.poll(pollDuration);
@@ -548,13 +552,19 @@ private void submitConsumerWithObserver(
548552
private void progressWatermarkOnEmptyPartitions(
549553
ElementConsumer<Object, Object> consumer,
550554
Map<TopicPartition, Integer> emptyPolls,
555+
Map<TopicPartition, Long> endOffsets,
556+
Map<TopicPartition, Long> polledOffsets,
551557
Map<TopicPartition, Integer> topicPartitionToId,
552558
AtomicReference<PartitionedWatermarkEstimator> watermarkEstimator) {
553559

554560
int partitions = emptyPolls.size();
555561
List<TopicPartition> idlingPartitions =
556562
emptyPolls.entrySet().stream()
557563
.filter(e -> e.getValue() >= partitions)
564+
.filter(
565+
e ->
566+
MoreObjects.firstNonNull(polledOffsets.get(e.getKey()), -1L) + 1
567+
>= MoreObjects.firstNonNull(endOffsets.get(e.getKey()), 0L))
558568
.map(Entry::getKey)
559569
.collect(Collectors.toList());
560570

@@ -624,7 +634,7 @@ private void logConsumerWatermark(
624634

625635
if (log.isDebugEnabled()) {
626636
log.debug(
627-
"Current watermark of consumer name {} with offsets {} " + "on {} poll'd records is {}",
637+
"Current watermark of consumer name {} with offsets {} on {} poll'd records is {}",
628638
name,
629639
offsets,
630640
polledCount,
@@ -723,14 +733,6 @@ public void waitUntilReady() throws InterruptedException {
723733
};
724734
}
725735

726-
private Map<TopicPartition, Long> findNonEmptyEndOffsets(final KafkaConsumer<?, ?> kafka) {
727-
Set<TopicPartition> assignment = kafka.assignment();
728-
Map<TopicPartition, Long> beginning = kafka.beginningOffsets(assignment);
729-
return kafka.endOffsets(assignment).entrySet().stream()
730-
.filter(entry -> beginning.get(entry.getKey()) < entry.getValue())
731-
.collect(toMap(Map.Entry::getKey, Map.Entry::getValue));
732-
}
733-
734736
private KafkaConsumer<Object, Object> createConsumer() {
735737
return createConsumer(UUID.randomUUID().toString(), null, null, Position.NEWEST);
736738
}
@@ -883,6 +885,7 @@ private ConsumerRebalanceListener listener(
883885
Map<TopicPartition, Integer> topicPartitionToId,
884886
Map<TopicPartition, Integer> emptyPolls,
885887
Map<TopicPartition, Long> polledOffsets,
888+
Map<TopicPartition, Long> endOffsets,
886889
AtomicReference<PartitionedWatermarkEstimator> watermarkEstimator) {
887890

888891
return new ConsumerRebalanceListener() {
@@ -901,6 +904,9 @@ public void onPartitionsAssigned(Collection<TopicPartition> parts) {
901904
topicPartitionToId.clear();
902905
AtomicInteger id = new AtomicInteger();
903906
emptyPolls.clear();
907+
polledOffsets.clear();
908+
endOffsets.clear();
909+
Optional.ofNullable(kafka.get()).ifPresent(k -> endOffsets.putAll(k.endOffsets(parts)));
904910

905911
currentlyAssigned.forEach(p -> topicPartitionToId.put(p, id.getAndIncrement()));
906912

@@ -921,13 +927,14 @@ public void onPartitionsAssigned(Collection<TopicPartition> parts) {
921927
name != null
922928
? getCommittedTopicOffsets(currentlyAssigned, c)
923929
: getCurrentTopicOffsets(currentlyAssigned, c);
924-
polledOffsets.clear();
925-
newOffsets.forEach(
926-
o ->
927-
polledOffsets.put(
928-
new TopicPartition(
929-
o.getPartition().getTopic(), o.getPartition().getPartition()),
930-
o.getOffset() - 1));
930+
newOffsets.stream()
931+
.filter(o -> o.getOffset() > 0)
932+
.forEach(
933+
o ->
934+
polledOffsets.put(
935+
new TopicPartition(
936+
o.getPartition().getTopic(), o.getPartition().getPartition()),
937+
o.getOffset() - 1));
931938
consumer.onAssign(c, newOffsets);
932939
});
933940
}

direct/io-kafka/src/test/java/cz/o2/proxima/direct/io/kafka/LocalKafkaCommitLogDescriptor.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -337,7 +337,7 @@ private Map<TopicPartition, Long> getEndOffsets(String name, Collection<TopicPar
337337
private <K, V> ConsumerRecords<K, V> pollConsumer(
338338
ConsumerGroup group,
339339
long period,
340-
ConsumerId consumerId,
340+
final ConsumerId consumerId,
341341
ElementSerializer<K, V> serializer,
342342
@Nullable ConsumerRebalanceListener listener)
343343
throws InterruptedException {
@@ -388,7 +388,6 @@ private <K, V> ConsumerRecords<K, V> pollConsumer(
388388
Optional.ofNullable(offsets.get(partition))
389389
.orElse(getCommittedOffset(name, part.getId()));
390390
log.trace("Partition {} has last {}, reading from {}", partition, last, off);
391-
392391
while (off < last && maxToPoll-- > 0) {
393392
if (off >= 0) {
394393
records.add(toConsumerRecord(partitionData.get(off), serializer, part.getId(), off));

direct/io-kafka/src/test/java/cz/o2/proxima/direct/io/kafka/LocalKafkaCommitLogDescriptorTest.java

Lines changed: 15 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,7 @@
7979
import java.util.concurrent.ExecutorService;
8080
import java.util.concurrent.Executors;
8181
import java.util.concurrent.TimeUnit;
82+
import java.util.concurrent.atomic.AtomicBoolean;
8283
import java.util.concurrent.atomic.AtomicInteger;
8384
import java.util.concurrent.atomic.AtomicLong;
8485
import java.util.concurrent.atomic.AtomicReference;
@@ -1124,15 +1125,16 @@ void testPollFromNConsumersMovesWatermark(
11241125
final long now = System.currentTimeMillis();
11251126
CountDownLatch latch = new CountDownLatch(numObservers);
11261127
Map<CommitLogObserver, Long> observerWatermarks = new ConcurrentHashMap<>();
1127-
AtomicInteger readyObservers = new AtomicInteger();
1128+
Map<Integer, Integer> idles = new ConcurrentHashMap<>();
11281129
for (int i = 0; i < numObservers; i++) {
1130+
int id = i;
11291131
reader
11301132
.observe(
11311133
"test-" + expectMoved,
11321134
Position.OLDEST,
11331135
new CommitLogObserver() {
11341136

1135-
boolean confimedLatch = false;
1137+
boolean confirmedLatch = false;
11361138

11371139
@Override
11381140
public boolean onNext(StreamElement element, OnNextContext context) {
@@ -1147,28 +1149,24 @@ public boolean onError(Throwable error) {
11471149

11481150
@Override
11491151
public void onIdle(OnIdleContext context) {
1150-
if (readyObservers.get() == numObservers) {
1151-
Long oldWatermark = observerWatermarks.put(this, context.getWatermark());
1152-
boolean hasWatermark =
1153-
MoreObjects.firstNonNull(oldWatermark, Long.MIN_VALUE) > 0;
1154-
if ((!expectMoved || !hasWatermark && context.getWatermark() > 0)
1155-
&& !confimedLatch) {
1156-
1157-
confimedLatch = true;
1152+
idles.compute(id, (k, v) -> MoreObjects.firstNonNull(v, 0) + 1);
1153+
if (context.getWatermark() > 0) {
1154+
observerWatermarks.put(this, context.getWatermark());
1155+
if ((!expectMoved || context.getWatermark() > 0) && !confirmedLatch) {
1156+
confirmedLatch = true;
11581157
latch.countDown();
11591158
}
11601159
}
11611160
}
11621161
})
11631162
.waitUntilReady();
1164-
readyObservers.incrementAndGet();
11651163
}
11661164

11671165
assertTrue(
11681166
String.format(
1169-
"Timeout, readyObservers = %d, observerWatermarks == %s, numObservers = %d",
1170-
readyObservers.get(), observerWatermarks, numObservers),
1171-
latch.await(10, TimeUnit.SECONDS));
1167+
"Timeout, observerWatermarks = %s, numObservers = %d",
1168+
observerWatermarks, numObservers),
1169+
latch.await(30, TimeUnit.SECONDS));
11721170

11731171
assertEquals(numObservers, observerWatermarks.size());
11741172
long watermark = observerWatermarks.values().stream().min(Long::compare).orElse(Long.MIN_VALUE);
@@ -1511,10 +1509,11 @@ public boolean onNext(StreamElement element, OnNextContext confirm) {
15111509
}
15121510
});
15131511
reader.observe("test", observer);
1512+
AtomicBoolean finished = new AtomicBoolean();
15141513
Executors.newCachedThreadPool()
15151514
.execute(
15161515
() -> {
1517-
while (true) {
1516+
while (!finished.get()) {
15181517
try {
15191518
TimeUnit.MILLISECONDS.sleep(100);
15201519
} catch (InterruptedException ex) {
@@ -1525,6 +1524,7 @@ public boolean onNext(StreamElement element, OnNextContext confirm) {
15251524
});
15261525
latch.await();
15271526
assertEquals(3, restarts.get());
1527+
finished.set(true);
15281528
}
15291529

15301530
@Test(timeout = 10000)

0 commit comments

Comments
 (0)