private void onCommit(ReceiverRecord<?, ?> record, CountDownLatch commitLatch, long[] committedOffsets) { committedOffsets[record.partition()] = record.offset() + 1; commitLatch.countDown(); }
.subscribe(partitionFlux -> subscribeDisposables.add(partitionFlux.publishOn(scheduler).subscribe(record -> { int partition = record.partition(); String current = Thread.currentThread().getName() + ":" + record.offset(); String inProgress = inProgressMap.putIfAbsent(partition, current); if (inProgress != null) {
@Test public void manualCommitSync() throws Exception { int count = 10; CountDownLatch commitLatch = new CountDownLatch(count); long[] committedOffsets = new long[partitions]; for (int i = 0; i < committedOffsets.length; i++) committedOffsets[i] = 0; receiverOptions = receiverOptions.commitInterval(Duration.ZERO).commitBatchSize(0); KafkaReceiver<Integer, String> receiver = createReceiver(); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = receiver.receive() .delayUntil(record -> { assertEquals(committedOffsets[record.partition()], record.offset()); return record.receiverOffset().commit() .doOnSuccess(i -> onCommit(record, commitLatch, committedOffsets)); }) .doOnError(e -> log.error("KafkaFlux exception", e)); sendAndWaitForMessages(kafkaFlux, count); checkCommitCallbacks(commitLatch, committedOffsets); }
@Test public void manualCommitSyncNoPoll() throws Exception { CountDownLatch commitLatch = new CountDownLatch(1); long[] committedOffsets = new long[partitions]; for (int i = 0; i < committedOffsets.length; i++) committedOffsets[i] = 0; receiverOptions = receiverOptions.commitInterval(Duration.ZERO).commitBatchSize(0) .consumerProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KafkaReceiver<Integer, String> receiver = createReceiver(); Flux<ReceiverRecord<Integer, String>> inboundFlux = receiver.receive() .doOnNext(record -> onReceive(record)); sendMessages(0, 10); StepVerifier.create( inboundFlux.take(1) .concatMap(record -> { assertEquals(committedOffsets[record.partition()], record.offset()); return record.receiverOffset() .commit() .doOnSuccess(i -> onCommit(record, commitLatch, committedOffsets)) .then(Mono.just(record)); }), 1) .expectNextCount(1) .expectComplete() .verify(Duration.ofSeconds(receiveTimeoutMillis)); checkCommitCallbacks(commitLatch, committedOffsets); }
@Test public void manualCommitAsyncNoPoll() throws Exception { CountDownLatch commitLatch = new CountDownLatch(1); long[] committedOffsets = new long[partitions]; for (int i = 0; i < committedOffsets.length; i++) committedOffsets[i] = 0; receiverOptions = receiverOptions.commitInterval(Duration.ZERO).commitBatchSize(0) .consumerProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KafkaReceiver<Integer, String> receiver = createReceiver(); Flux<ReceiverRecord<Integer, String>> inboundFlux = receiver.receive() .doOnNext(record -> onReceive(record)); sendMessages(0, 10); StepVerifier.create(inboundFlux, 1) .consumeNextWith(record -> { assertEquals(committedOffsets[record.partition()], record.offset()); record.receiverOffset().commit() .doOnSuccess(i -> onCommit(record, commitLatch, committedOffsets)) .subscribe(); }) .thenCancel() .verify(Duration.ofSeconds(receiveTimeoutMillis)); checkCommitCallbacks(commitLatch, committedOffsets); } @Test