private Disposable sendAndWaitForMessages(Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux, int count) throws Exception { CountDownLatch receiveLatch = new CountDownLatch(count); Disposable disposable = subscribe(kafkaFlux, receiveLatch); sendMessages(0, count); waitForMessages(receiveLatch); return disposable; }
private void sendReceiveWithSendDelay(Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux, Duration sendDelay, int startIndex, int count) throws Exception { CountDownLatch latch = new CountDownLatch(count); subscribe(kafkaFlux, latch); Thread.sleep(sendDelay.toMillis()); sendMessages(startIndex, count); waitForMessages(latch); checkConsumedMessages(startIndex, count); }
private void sendReceive(Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux, int sendStartIndex, int sendCount, int receiveStartIndex, int receiveCount) throws Exception { CountDownLatch latch = new CountDownLatch(receiveCount); subscribe(kafkaFlux, latch); if (sendCount > 0) sendMessages(sendStartIndex, sendCount); waitForMessages(latch); checkConsumedMessages(receiveStartIndex, receiveCount); }
private void sendReceiveWithRedelivery(Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux, int sendStartIndex, int sendCount, int minRedelivered, int maxRedelivered) throws Exception { int maybeRedelivered = maxRedelivered - minRedelivered; CountDownLatch latch = new CountDownLatch(sendCount + maxRedelivered); subscribe(kafkaFlux, latch); sendMessages(sendStartIndex, sendCount); // Countdown the latch manually for messages that may or may not be redelivered on each partition for (int i = 0; i < partitions; i++) { TestUtils.waitUntil("Messages not received on partition " + i, null, list -> list.size() > 0, receivedMessages.get(i), Duration.ofMillis(receiveTimeoutMillis)); } int minReceiveIndex = sendStartIndex - minRedelivered; for (int i = minReceiveIndex - maybeRedelivered; i < minReceiveIndex; i++) { int partition = i % partitions; if (receivedMessages.get(partition).get(0) > i) latch.countDown(); } // Wait for messages, redelivered as well as those sent here waitForMessages(latch); // Within the range including redelivered, check that all messages were delivered. for (int i = 0; i < partitions; i++) { List<Integer> received = receivedMessages.get(i); int receiveStartIndex = received.get(0); int receiveEndIndex = received.get(received.size() - 1); checkConsumedMessages(i, receiveStartIndex, receiveEndIndex); } }
private void testManualCommitRetry(boolean retriableException) throws Exception { int count = 1; int failureCount = 2; Semaphore receiveSemaphore = new Semaphore(1 - count); Semaphore commitSuccessSemaphore = new Semaphore(0); Semaphore commitFailureSemaphore = new Semaphore(0); receiverOptions = receiverOptions.commitInterval(Duration.ZERO).commitBatchSize(0); KafkaReceiver<Integer, String> receiver = createReceiver(); TestableReceiver testableReceiver = new TestableReceiver(receiver); Flux<? extends ConsumerRecord<Integer, String>> flux = testableReceiver .receiveWithManualCommitFailures(retriableException, failureCount, receiveSemaphore, commitSuccessSemaphore, commitFailureSemaphore); subscribe(flux, new CountDownLatch(count)); sendMessages(1, count); assertTrue("Did not receive messages", receiveSemaphore.tryAcquire(receiveTimeoutMillis, TimeUnit.MILLISECONDS)); assertTrue("Commit did not succeed after retry", commitSuccessSemaphore.tryAcquire(receiveTimeoutMillis, TimeUnit.MILLISECONDS)); assertEquals(failureCount, commitFailureSemaphore.availablePermits()); }
@Test public void sendTransactionalReadUncommitted() throws Exception { receiverOptions = receiverOptions.consumerProperty(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_uncommitted"); int count = 100; CountDownLatch latch1 = new CountDownLatch(count); CountDownLatch latch2 = new CountDownLatch(count * 2); CountDownLatch latch3 = new CountDownLatch(count * 3); subscribe(createReceiver().receive(), latch1, latch2, latch3); sendMessages(0, count); waitForMessages(latch1); // non-transactional messages received KafkaSender<Integer, String> txSender = createTransactionalSender(); txSender.sendTransactionally(Flux.just(createSenderRecords(count, count, true))) .then().block(Duration.ofSeconds(receiveTimeoutMillis)); waitForMessages(latch2); // transactional messages received before commit sendMessages(count * 2, count); waitForMessages(latch3); checkConsumedMessages(0, count * 3); }
@Test public void publishFromEventScheduler() throws Exception { receiverOptions = receiverOptions .schedulerSupplier(Schedulers::immediate) .addAssignListener(this::onPartitionsAssigned) .subscription(Collections.singletonList(topic)); KafkaReceiver<Integer, String> receiver = KafkaReceiver.create(receiverOptions); AtomicReference<String> publishingThreadName = new AtomicReference<>(); CountDownLatch receiveLatch = new CountDownLatch(1); Disposable disposable = receiver.receive() .doOnNext(record -> { publishingThreadName.set(Thread.currentThread().getName()); record.receiverOffset().acknowledge(); receiveLatch.countDown(); }) .subscribe(); subscribeDisposables.add(disposable); waitFoPartitionAssignment(); sendMessages(0, 1); waitForMessages(receiveLatch); assertNotNull(publishingThreadName.get()); assertTrue(publishingThreadName.get().startsWith("reactive-kafka-")); }
@Test public void seekToBeginning() throws Exception { int count = 10; sendMessages(0, count); receiverOptions = receiverOptions .addAssignListener(this::seekToBeginning) .subscription(Collections.singletonList(topic)); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = KafkaReceiver.create(receiverOptions) .receive(); sendReceive(kafkaFlux, count, count, 0, count * 2); }
@Test public void publishFromCustomScheduler() throws Exception { String schedulerName = "custom-scheduler"; Scheduler scheduler = Schedulers.newElastic(schedulerName); receiverOptions = receiverOptions .schedulerSupplier(() -> scheduler) .addAssignListener(this::onPartitionsAssigned) .subscription(Collections.singletonList(topic)); KafkaReceiver<Integer, String> receiver = KafkaReceiver.create(receiverOptions); AtomicReference<String> publishingThreadName = new AtomicReference<>(); CountDownLatch receiveLatch = new CountDownLatch(1); Disposable disposable = receiver.receive() .doOnNext(record -> { publishingThreadName.set(Thread.currentThread().getName()); record.receiverOffset().acknowledge(); receiveLatch.countDown(); }) .subscribe(); subscribeDisposables.add(scheduler); subscribeDisposables.add(disposable); waitFoPartitionAssignment(); sendMessages(0, 1); waitForMessages(receiveLatch); assertNotNull(publishingThreadName.get()); assertTrue(publishingThreadName.get().startsWith(schedulerName)); }
@Test public void offsetResetLatest() throws Exception { int count = 10; sendMessages(0, count); receiverOptions = receiverOptions .consumerProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") .addAssignListener(partitions -> assignSemaphore.release()); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = createReceiver() .receive() .doOnNext(record -> onReceive(record)); StepVerifier.create(kafkaFlux) .then(() -> assignSemaphore.acquireUninterruptibly()) .expectNoEvent(Duration.ofMillis(100)) .then(() -> sendMessages(count, count)) .expectNextCount(count) .thenCancel() .verify(Duration.ofSeconds(receiveTimeoutMillis)); checkConsumedMessages(count, count); }
@Test public void seekToEnd() throws Exception { int count = 10; sendMessages(0, count); receiverOptions = receiverOptions .addAssignListener(partitions -> { for (ReceiverPartition partition : partitions) partition.seekToEnd(); onPartitionsAssigned(partitions); }) .subscription(Collections.singletonList(topic)); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = KafkaReceiver.create(receiverOptions) .receive(); sendReceiveWithSendDelay(kafkaFlux, Duration.ofMillis(100), count, count); }
@Test public void seekToOffset() throws Exception { int count = 10; sendMessages(0, count); receiverOptions = receiverOptions .addAssignListener(partitions -> { onPartitionsAssigned(partitions); for (ReceiverPartition partition : partitions) partition.seek(1); }) .subscription(Collections.singletonList(topic)); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = KafkaReceiver.create(receiverOptions) .receive() .doOnError(e -> log.error("KafkaFlux exception", e)); sendReceive(kafkaFlux, count, count, partitions, count * 2 - partitions); }
@Test public void manualCommitFailure() throws Exception { int count = 1; AtomicBoolean commitSuccess = new AtomicBoolean(); Semaphore commitErrorSemaphore = new Semaphore(0); receiverOptions = receiverOptions.commitInterval(Duration.ZERO).commitBatchSize(0); KafkaReceiver<Integer, String> receiver = createReceiver(); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = receiver.receive() .doOnNext(record -> { ReceiverOffset offset = record.receiverOffset(); TestableReceiver.setNonExistentPartition(offset); record.receiverOffset().acknowledge(); record.receiverOffset().commit() .doOnError(e -> commitErrorSemaphore.release()) .doOnSuccess(i -> commitSuccess.set(true)) .subscribe(); }) .doOnError(e -> log.error("KafkaFlux exception", e)); subscribe(kafkaFlux, new CountDownLatch(count)); sendMessages(1, count); assertTrue("Commit error callback not invoked", commitErrorSemaphore.tryAcquire(receiveTimeoutMillis, TimeUnit.MILLISECONDS)); assertFalse("Commit of non existent topic succeeded", commitSuccess.get()); }
@Test public void messageProcessorFailure() throws Exception { int count = 200; int successfulReceives = 100; CountDownLatch receiveLatch = new CountDownLatch(successfulReceives + 1); receiverOptions = receiverOptions .addAssignListener(this::onPartitionsAssigned) .subscription(Collections.singletonList(topic)); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = KafkaReceiver.create(receiverOptions) .receive() .publishOn(Schedulers.single()) .doOnNext(record -> { receiveLatch.countDown(); if (receiveLatch.getCount() == 0) throw new RuntimeException("Test exception"); record.receiverOffset().acknowledge(); }); CountDownLatch latch = new CountDownLatch(successfulReceives); subscribe(kafkaFlux, latch); sendMessages(0, count); waitForMessages(latch); TestUtils.sleep(100); assertEquals(successfulReceives, count(receivedMessages)); }
@Test public void sendNonTransactionalReadCommitted() throws Exception { receiverOptions = receiverOptions.consumerProperty(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); int count = 100; CountDownLatch latch1 = new CountDownLatch(count); CountDownLatch latch2 = new CountDownLatch(count * 3); subscribe(createReceiver().receive(), latch1, latch2); sendMessages(0, count); waitForMessages(latch1); // non-transactional messages received if no commits pending checkConsumedMessages(0, count); KafkaSender<Integer, String> txSender = createTransactionalSender(); TransactionManager txn = txSender.transactionManager(); txn.begin() .thenMany(txSender.send(createSenderRecords(count, count, true))) .blockLast(Duration.ofSeconds(receiveTimeoutMillis)); sendMessages(count * 2, count); Thread.sleep(1000); assertEquals(count * 2, latch2.getCount()); // non-transactional and transactional messages not received while commit pending txn.commit().subscribe(); waitForMessages(latch2); checkConsumedMessages(0, count * 3); }
@Test public void resumeAfterFailure() throws Exception { int count = 20; CountDownLatch receiveLatch = new CountDownLatch(count + 1); receiverOptions = receiverOptions.consumerProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") .addAssignListener(this::onPartitionsAssigned) .subscription(Collections.singletonList(topic)); KafkaReceiver<Integer, String> receiver = KafkaReceiver.create(receiverOptions); Consumer<ReceiverRecord<Integer, String>> onNext = record -> { receiveLatch.countDown(); onReceive(record); log.info("onNext {}", record.value()); if (receiveLatch.getCount() == 10) throw new RuntimeException("Test exception"); record.receiverOffset().acknowledge(); }; Disposable disposable = receiver.receive() .doOnNext(onNext) .onErrorResume(e -> receiver.receive().doOnNext(onNext)) .subscribe(); subscribeDisposables.add(disposable); waitFoPartitionAssignment(); sendMessages(0, count); waitForMessages(receiveLatch); }
@Test public void manualCommitRecordAsync() throws Exception { int count = 10; CountDownLatch commitLatch = new CountDownLatch(count); long[] committedOffsets = new long[partitions]; receiverOptions = receiverOptions .commitInterval(Duration.ZERO) .commitBatchSize(0) .addAssignListener(this::seekToBeginning) .subscription(Collections.singletonList(topic)); Flux<ReceiverRecord<Integer, String>> kafkaFlux = KafkaReceiver.create(receiverOptions) .receive() .doOnNext(record -> record.receiverOffset() .commit() .doOnSuccess(i -> onCommit(record, commitLatch, committedOffsets)) .doOnError(e -> log.error("Commit exception", e)) .subscribe()); subscribe(kafkaFlux, new CountDownLatch(count)); sendMessages(0, count); checkCommitCallbacks(commitLatch, committedOffsets); }
@Test public void abortTransaction() throws Exception { receiverOptions = receiverOptions.consumerProperty(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); Flux<? extends ConsumerRecord<Integer, String>> kafkaFlux = createReceiver().receive(); int count = 100; CountDownLatch latch1 = new CountDownLatch(count); CountDownLatch latch2 = new CountDownLatch(count * 2); subscribe(kafkaFlux, latch1, latch2); KafkaSender<Integer, String> txSender = createTransactionalSender(); txSender.transactionManager().begin() .thenMany(txSender.send(createSenderRecords(0, count, false))) .then(txSender.transactionManager().abort()) .then().block(Duration.ofSeconds(receiveTimeoutMillis)); sendMessages(count, count); waitForMessages(latch1); // non-transactional messages received if no commits pending checkConsumedMessages(count, count); txSender.sendTransactionally(Flux.just(createSenderRecords(count * 2, count, true))) .then().subscribe(); waitForMessages(latch2); checkConsumedMessages(count, count * 3); }
@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