Skip to content

Commit

Permalink
Commit offsets at specified intervals (knative-extensions#1405)
Browse files Browse the repository at this point in the history
* Commit offset at a specified interval

Signed-off-by: Pierangelo Di Pilato <pdipilat@redhat.com>

* Add reset logic to handle long large offsets

Signed-off-by: Pierangelo Di Pilato <pdipilat@redhat.com>

* Use final

Co-authored-by: Matthias Wessendorf <mwessend@redhat.com>

Co-authored-by: Matthias Wessendorf <mwessend@redhat.com>
  • Loading branch information
2 people authored and knative-prow-robot committed Nov 10, 2021
1 parent ea1a02f commit 0e6b589
Show file tree
Hide file tree
Showing 13 changed files with 353 additions and 582 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import io.vertx.core.AsyncResult;
import io.vertx.core.Future;
import io.vertx.core.Handler;
import io.vertx.core.Vertx;
import io.vertx.kafka.client.common.PartitionInfo;
import io.vertx.kafka.client.common.TopicPartition;
import io.vertx.kafka.client.consumer.KafkaConsumerRecord;
Expand All @@ -27,11 +28,6 @@
import io.vertx.kafka.client.consumer.OffsetAndMetadata;
import io.vertx.kafka.client.consumer.OffsetAndTimestamp;
import io.vertx.kafka.client.consumer.impl.KafkaConsumerRecordImpl;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.openjdk.jmh.annotations.Benchmark;
Expand All @@ -41,6 +37,12 @@
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.infra.Blackhole;

import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;

public class UnorderedOffsetManagerBenchmark {

@State(Scope.Thread)
Expand Down Expand Up @@ -71,96 +73,70 @@ public void doSetup() {

@Benchmark
public void benchmarkReverseOrder(RecordsState recordsState, Blackhole blackhole) {
UnorderedOffsetManager offsetManager = new UnorderedOffsetManager(new MockKafkaConsumer(), null);
final OffsetManager offsetManager = new OffsetManager(Vertx.vertx(), new MockKafkaConsumer(), null, 10000L);

int partitions = 100;
for (int partition = 0; partition < partitions; partition++) {
blackhole.consume(
offsetManager.recordReceived(recordsState.records[partition][0])
);
offsetManager.recordReceived(recordsState.records[partition][0]);
}

for (int offset = 9_999; offset > 0; offset--) {
for (int partition = 0; partition < partitions; partition++) {
blackhole.consume(
offsetManager.recordReceived(recordsState.records[partition][offset])
);
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[partition][offset])
);
offsetManager.recordReceived(recordsState.records[partition][offset]);
offsetManager.successfullySentToSubscriber(recordsState.records[partition][offset]);
}
}

for (int partition = 0; partition < partitions; partition++) {
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[partition][0])
);
offsetManager.successfullySentToSubscriber(recordsState.records[partition][0]);
}
}

@Benchmark
public void benchmarkOrdered(RecordsState recordsState, Blackhole blackhole) {
UnorderedOffsetManager offsetManager = new UnorderedOffsetManager(new MockKafkaConsumer(), null);
OffsetManager offsetManager = new OffsetManager(Vertx.vertx(), new MockKafkaConsumer(), null, 10000L);
int partitions = 100;

for (int offset = 0; offset < 10_000; offset++) {
for (int partition = 0; partition < partitions; partition++) {
blackhole.consume(
offsetManager.recordReceived(recordsState.records[partition][offset])
);
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[partition][offset])
);
offsetManager.recordReceived(recordsState.records[partition][offset]);
offsetManager.successfullySentToSubscriber(recordsState.records[partition][offset]);
}
}
}

@Benchmark
public void benchmarkRealisticCase(RecordsState recordsState, Blackhole blackhole) {
UnorderedOffsetManager offsetManager = new UnorderedOffsetManager(new MockKafkaConsumer(), null);
OffsetManager offsetManager = new OffsetManager(Vertx.vertx(), new MockKafkaConsumer(), null, 10000L);
int partitions = 10;

for (int partition = 0; partition < partitions; partition++) {
blackhole.consume(
offsetManager.recordReceived(recordsState.records[partition][0])
);
offsetManager.recordReceived(recordsState.records[partition][0]);
}

for (int partition = 0; partition < partitions; partition++) {
for (int offset : new int[] {5, 2, 0, 7, 1, 3, 4, 6}) {
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[partition][offset])
);
for (int offset : new int[]{5, 2, 0, 7, 1, 3, 4, 6}) {
offsetManager.successfullySentToSubscriber(recordsState.records[partition][offset]);
}
}
}

@Benchmark
public void benchmarkMixedABit(RecordsState recordsState, Blackhole blackhole) {
UnorderedOffsetManager offsetManager = new UnorderedOffsetManager(new MockKafkaConsumer(), null);
OffsetManager offsetManager = new OffsetManager(Vertx.vertx(), new MockKafkaConsumer(), null, 10000L);
int partitions = 4;

for (int partition = 0; partition < partitions; partition++) {
blackhole.consume(
offsetManager.recordReceived(recordsState.records[partition][0])
);
offsetManager.recordReceived(recordsState.records[partition][0]);
}

for (int i = 0; i < 120; i++) {
// This will commit in the following order:
// 1 0 3 2 5 4 ...
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[2][i % 2 == 0 ? i + 1 : i - 1])
);
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[1][i % 2 == 0 ? i + 1 : i - 1])
);
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[0][i % 2 == 0 ? i + 1 : i - 1])
);
blackhole.consume(
offsetManager.successfullySentToSubscriber(recordsState.records[3][i % 2 == 0 ? i + 1 : i - 1])
);
offsetManager.successfullySentToSubscriber(recordsState.records[2][i % 2 == 0 ? i + 1 : i - 1]);
offsetManager.successfullySentToSubscriber(recordsState.records[1][i % 2 == 0 ? i + 1 : i - 1]);
offsetManager.successfullySentToSubscriber(recordsState.records[0][i % 2 == 0 ? i + 1 : i - 1]);
offsetManager.successfullySentToSubscriber(recordsState.records[3][i % 2 == 0 ? i + 1 : i - 1]);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,34 +28,34 @@ public interface RecordDispatcherListener {
*
* @param record record received.
*/
Future<Void> recordReceived(KafkaConsumerRecord<?, ?> record);
void recordReceived(KafkaConsumerRecord<?, ?> record);

/**
* The given record cannot be delivered to dead letter sink.
*
* @param record record undeliverable to dead letter sink.
* @param ex exception occurred.
*/
Future<Void> failedToSendToDeadLetterSink(KafkaConsumerRecord<?, ?> record, Throwable ex);
void failedToSendToDeadLetterSink(KafkaConsumerRecord<?, ?> record, Throwable ex);

/**
* The given event doesn't pass the filter.
*
* @param record record discarded.
*/
Future<Void> recordDiscarded(KafkaConsumerRecord<?, ?> record);
void recordDiscarded(KafkaConsumerRecord<?, ?> record);

/**
* The given record has been successfully sent to subscriber.
*
* @param record record sent to subscriber.
*/
Future<Void> successfullySentToSubscriber(KafkaConsumerRecord<?, ?> record);
void successfullySentToSubscriber(KafkaConsumerRecord<?, ?> record);

/**
* The given record has been successfully sent to dead letter sink.
*
* @param record record sent to dead letter sink.
*/
Future<Void> successfullySentToDeadLetterSink(KafkaConsumerRecord<?, ?> record);
void successfullySentToDeadLetterSink(KafkaConsumerRecord<?, ?> record);
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,10 @@
import io.vertx.core.Vertx;
import io.vertx.kafka.client.common.tracing.ConsumerTracer;
import io.vertx.kafka.client.consumer.KafkaConsumerRecord;

import java.util.Objects;
import java.util.function.Function;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -139,16 +141,13 @@ private void onRecordReceived(final KafkaConsumerRecord<String, CloudEvent> reco
}
}

recordDispatcherListener.recordReceived(record)
.onSuccess(v -> {
// Execute filtering
if (filter.test(record.value())) {
onFilterMatching(record, finalProm);
} else {
onFilterNotMatching(record, finalProm);
}
})
.onFailure(finalProm::fail); // This should really never happen
recordDispatcherListener.recordReceived(record);
// Execute filtering
if (filter.test(record.value())) {
onFilterMatching(record, finalProm);
} else {
onFilterNotMatching(record, finalProm);
}
}

private void onFilterMatching(final KafkaConsumerRecord<String, CloudEvent> record, final Promise<Void> finalProm) {
Expand All @@ -161,15 +160,15 @@ private void onFilterMatching(final KafkaConsumerRecord<String, CloudEvent> reco
private void onFilterNotMatching(final KafkaConsumerRecord<String, CloudEvent> record,
final Promise<Void> finalProm) {
logDebug("Record doesn't match filtering", record);
recordDispatcherListener.recordDiscarded(record)
.onComplete(finalProm);
recordDispatcherListener.recordDiscarded(record);
finalProm.complete();
}

private void onSubscriberSuccess(final KafkaConsumerRecord<String, CloudEvent> record,
final Promise<Void> finalProm) {
logDebug("Successfully sent event to subscriber", record);
recordDispatcherListener.successfullySentToSubscriber(record)
.onComplete(finalProm);
recordDispatcherListener.successfullySentToSubscriber(record);
finalProm.complete();
}

private void onSubscriberFailure(final KafkaConsumerRecord<String, CloudEvent> record,
Expand All @@ -182,15 +181,15 @@ private void onSubscriberFailure(final KafkaConsumerRecord<String, CloudEvent> r
private void onDeadLetterSinkSuccess(final KafkaConsumerRecord<String, CloudEvent> record,
final Promise<Void> finalProm) {
logDebug("Successfully sent event to the dead letter sink", record);
recordDispatcherListener.successfullySentToDeadLetterSink(record)
.onComplete(finalProm);
recordDispatcherListener.successfullySentToDeadLetterSink(record);
finalProm.complete();
}


private void onDeadLetterSinkFailure(final KafkaConsumerRecord<String, CloudEvent> record, final Throwable exception,
final Promise<Void> finalProm) {
recordDispatcherListener.failedToSendToDeadLetterSink(record, exception)
.onComplete(finalProm);
recordDispatcherListener.failedToSendToDeadLetterSink(record, exception);
finalProm.complete();
}

private static Function<KafkaConsumerRecord<String, CloudEvent>, Future<Void>> composeSenderAndSinkHandler(
Expand Down
Loading

0 comments on commit 0e6b589

Please sign in to comment.