From ccdce2c429136f2a9898ebb99bf027267203b5a1 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Fri, 17 Nov 2023 23:14:41 -0500 Subject: [PATCH 1/9] Step 1 to fix a bug where commits throw and we can't recover. Catch exceptions from the commit and reset all the current commit points so that we don't begin to cause an out-of-order sequence. Signed-off-by: Greg Schohn --- .../replay/TrafficCaptureSourceFactory.java | 4 +- .../datatypes/PojoTrafficStreamKey.java | 2 + .../replay/kafka/KafkaProtobufConsumer.java | 68 ++++++++++++----- .../traffic/source/ITrafficCaptureSource.java | 14 ++++ .../KafkaRestartingTrafficReplayerTest.java | 6 +- .../KafkaProtobufConsumerLongTermTest.java | 75 +++++++++++++++++-- 6 files changed, 141 insertions(+), 28 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java index 68ea3b8a1..0d90f062e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java @@ -9,6 +9,7 @@ import java.io.FileInputStream; import java.io.IOException; +import java.time.Clock; import java.time.Duration; @Slf4j @@ -32,7 +33,8 @@ private TrafficCaptureSourceFactory() {} if (isKafkaActive) { return KafkaProtobufConsumer.buildKafkaConsumer(appParams.kafkaTrafficBrokers, appParams.kafkaTrafficTopic, - appParams.kafkaTrafficGroupId, appParams.kafkaTrafficEnableMSKAuth, appParams.kafkaTrafficPropertyFile, new KafkaBehavioralPolicy()); + appParams.kafkaTrafficGroupId, appParams.kafkaTrafficEnableMSKAuth, appParams.kafkaTrafficPropertyFile, + Clock.systemUTC(), new KafkaBehavioralPolicy()); } else if (isInputFileActive) { return new InputStreamOfTraffic(new FileInputStream(appParams.inputFilename)); } else { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java index ce32ab971..da4ce3ee1 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/datatypes/PojoTrafficStreamKey.java @@ -1,10 +1,12 @@ package org.opensearch.migrations.replay.datatypes; +import lombok.EqualsAndHashCode; import lombok.ToString; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; @ToString +@EqualsAndHashCode() public class PojoTrafficStreamKey implements ITrafficStreamKey { private final String nodeId; private final String connectionId; diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java index 2282121fd..e2c5e9d7d 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java @@ -1,6 +1,7 @@ package org.opensearch.migrations.replay.kafka; import com.google.protobuf.InvalidProtocolBufferException; +import lombok.EqualsAndHashCode; import lombok.NonNull; import lombok.ToString; import lombok.extern.slf4j.Slf4j; @@ -19,11 +20,14 @@ import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.slf4j.event.Level; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.time.Clock; import java.time.Duration; +import java.time.Instant; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -49,8 +53,7 @@ * may not be called for almost an hour. By design, we're not calling Kafka to pull * any more messages since we know that we don't have work to do for an hour. Shortly * after the hour of waiting begins, Kakfa will notice that this application is no - * longer calling poll and will kick the consumer out of the client group. Other - * consumers may connect, though they'll also be kicked out of the group shortly. + * longer calling poll and will kick the consumer out of the client group. * * See * ... @@ -60,15 +63,15 @@ * over its partitions. When this happens, you may see an offset commit failure (as * indicated by a CommitFailedException thrown from a call to commitSync())." * - * I believe that this can be mitigated, hopefully fully, by adding a keepAlive/do nothing - * call that the BlockingTrafficSource can use. That can be implemented in a source - * like this with Kafka by polling, then resetting the position on the stream if we - * aren't supposed to be reading new data. + * Since the Kafka client requires all calls to be made from the same thread, we can't + * simply run a background job to keep the client warm. We need the caller to touch + * this object periodically to keep the connection alive. */ @Slf4j public class KafkaProtobufConsumer implements ISimpleTrafficCaptureSource { @ToString(callSuper = true) + @EqualsAndHashCode(callSuper = true) private static class TrafficStreamKeyWithKafkaRecordId extends PojoTrafficStreamKey { private final int partition; private final long offset; @@ -119,6 +122,7 @@ Optional removeAndReturnNewHead(TrafficStreamKeyWithKafkaRecordId kafkaRec public static final Duration CONSUMER_POLL_TIMEOUT = Duration.ofSeconds(1); + private final Clock clock; private final Consumer kafkaConsumer; private final ConcurrentHashMap partitionToOffsetLifecycleTrackerMap; private final ConcurrentHashMap nextSetOfCommitsMap; @@ -128,16 +132,17 @@ Optional removeAndReturnNewHead(TrafficStreamKeyWithKafkaRecordId kafkaRec private final AtomicInteger trafficStreamsRead; public KafkaProtobufConsumer(Consumer kafkaConsumer, String topic) { - this(kafkaConsumer, topic, new KafkaBehavioralPolicy()); + this(kafkaConsumer, topic, Clock.systemUTC(), new KafkaBehavioralPolicy()); } public KafkaProtobufConsumer(Consumer kafkaConsumer, @NonNull String topic, - KafkaBehavioralPolicy behavioralPolicy) { + Clock clock, @NonNull KafkaBehavioralPolicy behavioralPolicy) { this.kafkaConsumer = kafkaConsumer; this.topic = topic; this.behavioralPolicy = behavioralPolicy; kafkaConsumer.subscribe(Collections.singleton(topic)); trafficStreamsRead = new AtomicInteger(); + this.clock = clock; partitionToOffsetLifecycleTrackerMap = new ConcurrentHashMap<>(); nextSetOfCommitsMap = new ConcurrentHashMap<>(); @@ -148,9 +153,12 @@ public static KafkaProtobufConsumer buildKafkaConsumer(@NonNull String brokers, @NonNull String groupId, boolean enableMSKAuth, String propertyFilePath, - KafkaBehavioralPolicy behavioralPolicy) throws IOException { + Clock clock, + @NonNull KafkaBehavioralPolicy behavioralPolicy) + throws IOException + { var kafkaProps = buildKafkaProperties(brokers, groupId, enableMSKAuth, propertyFilePath); - return new KafkaProtobufConsumer(new KafkaConsumer<>(kafkaProps), topic, behavioralPolicy); + return new KafkaProtobufConsumer(new KafkaConsumer<>(kafkaProps), topic, clock, behavioralPolicy); } public static Properties buildKafkaProperties(@NonNull String brokers, @@ -182,6 +190,16 @@ public static Properties buildKafkaProperties(@NonNull String brokers, return kafkaProps; } +// @Override +// public Optional maximumKeepAliveInterval() { +// return Optional.of(Duration.); +// } + + @Override + public Optional touch() { + return Optional.of(Instant.MAX); + } + @Override @SuppressWarnings("unchecked") public CompletableFuture> readNextTrafficStreamChunk() { @@ -191,7 +209,8 @@ public CompletableFuture> readNextTrafficStreamChunk public List readNextTrafficStreamSynchronously() { try { ConsumerRecords records; - records = safeCommitAndPollWithSwallowedRuntimeExceptions(); + safeCommitWithRetry(); + records = safePollWithSwallowedRuntimeExceptions(); Stream trafficStream = StreamSupport.stream(records.spliterator(), false) .map(kafkaRecord -> { try { @@ -226,19 +245,32 @@ public List readNextTrafficStreamSynchronously() { } } - private ConsumerRecords safeCommitAndPollWithSwallowedRuntimeExceptions() { - try { - synchronized (offsetLifecycleLock) { + private void safeCommitWithRetry() { + synchronized (offsetLifecycleLock) { + try { if (!nextSetOfCommitsMap.isEmpty()) { - log.atDebug().setMessage(()->"Committing "+nextSetOfCommitsMap).log(); + log.atDebug().setMessage(() -> "Committing " + nextSetOfCommitsMap).log(); kafkaConsumer.commitSync(nextSetOfCommitsMap); - log.atDebug().setMessage(()->"Done committing "+nextSetOfCommitsMap).log(); - nextSetOfCommitsMap.clear(); + log.atDebug().setMessage(() -> "Done committing " + nextSetOfCommitsMap).log(); } + } catch (RuntimeException e) { + log.atWarn().setCause(e) + .setMessage(() -> "Error while committing. Purging all commit points since another consumer " + + "may have already begun processing messages BEFORE those commits. Commits being " + + "discarded: " + nextSetOfCommitsMap.entrySet().stream() + .map(kvp -> kvp.getKey() + "->" + kvp.getValue()).collect(Collectors.joining(","))) + .log(); + } finally { + nextSetOfCommitsMap.clear(); } + } + } + private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { + try { var records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT); - log.atInfo().setMessage(()->"Kafka consumer poll has fetched "+records.count()+" records").log(); + log.atLevel(records.isEmpty()? Level.TRACE:Level.INFO) + .setMessage(()->"Kafka consumer poll has fetched "+records.count()+" records").log(); log.atDebug().setMessage(()->"All positions: {"+kafkaConsumer.assignment().stream() .map(tp->tp+": "+kafkaConsumer.position(tp)).collect(Collectors.joining(",")) + "}").log(); log.atDebug().setMessage(()->"All COMMITTED positions: {"+kafkaConsumer.assignment().stream() diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java index 43838a2a5..b0601e2bc 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java @@ -4,7 +4,10 @@ import java.io.Closeable; import java.io.IOException; +import java.time.Duration; +import java.time.Instant; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; public interface ITrafficCaptureSource extends Closeable { @@ -14,4 +17,15 @@ public interface ITrafficCaptureSource extends Closeable { default void commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException {} default void close() throws IOException {} + + /** + * Keep-alive call to be used by the BlockingTrafficSource to keep this connection alive if + * this is required. + * @return The time that the next call to touch() must be completed for this source to stay + * active. Empty indicates that touch() does not need to be called to keep the + * source active. + */ + default Optional touch() { + return Optional.empty(); + } } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java index 34d28db72..a9398bb81 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java @@ -90,7 +90,7 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { response->TestHttpServerContext.makeResponse(random, response)); var streamAndConsumer = TrafficStreamGenerator.generateStreamAndSumOfItsTransactions(testSize, randomize); var trafficStreams = streamAndConsumer.stream.collect(Collectors.toList()); - log.atInfo().setMessage(()->trafficStreams.stream().map(ts-> TrafficStreamUtils.summarizeTrafficStream(ts)) + log.atInfo().setMessage(()->trafficStreams.stream().map(TrafficStreamUtils::summarizeTrafficStream) .collect(Collectors.joining("\n"))).log(); loadStreamsToKafka(buildKafkaConsumer(), @@ -98,7 +98,7 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(streamAndConsumer.numHttpTransactions, httpServer.localhostEndpoint(), new CounterLimitedReceiverFactory(), () -> new SentinelSensingTrafficSource( - new KafkaProtobufConsumer(buildKafkaConsumer(), TEST_TOPIC_NAME, null))); + new KafkaProtobufConsumer(buildKafkaConsumer(), TEST_TOPIC_NAME))); log.error("done"); } @@ -174,7 +174,7 @@ Producer buildKafkaProducer() { throw Lombok.sneakyThrow(e); } }); - return () -> new KafkaProtobufConsumer(kafkaConsumer, TEST_TOPIC_NAME, null); + return () -> new KafkaProtobufConsumer(kafkaConsumer, TEST_TOPIC_NAME); } @SneakyThrows diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java index e8f478b04..c057ca314 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java @@ -2,6 +2,8 @@ import com.google.protobuf.ByteString; import com.google.protobuf.Timestamp; +import lombok.Lombok; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; @@ -12,7 +14,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.trafficcapture.protos.ReadObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; @@ -23,11 +25,16 @@ import java.nio.charset.StandardCharsets; import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; import java.util.Properties; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -41,6 +48,10 @@ public class KafkaProtobufConsumerLongTermTest { public static final String TEST_TRAFFIC_STREAM_ID_STRING = "TEST_TRAFFIC_STREAM_ID_STRING"; private static final String FAKE_READ_PACKET_DATA = "Fake pa"; public static final int PRODUCER_SLEEP_INTERVAL_MS = 100; + + public static final String MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY = "max.poll.interval.ms"; + public static final String HEARTBEAT_INTERVAL_MS_KEY = "heartbeat.interval.ms"; + @Container // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility private KafkaContainer embeddedKafkaBroker = @@ -89,6 +100,58 @@ private String getConnectionId(int i) { return TEST_TRAFFIC_STREAM_ID_STRING + "_" + i; } + @Test + @Tag("longTest") + public void testTimeoutsDontOccurForSlowPolls() throws Exception { + String testTopicName = "TEST_TOPIC"; + + var kafkaProducer = buildKafkaProducer(); + var sendCompleteCount = new AtomicInteger(0); + produceKafkaRecord(testTopicName, kafkaProducer, 0, sendCompleteCount).get(); + Assertions.assertEquals(1, sendCompleteCount.get()); + + var kafkaProperties = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), + TEST_GROUP_CONSUMER_ID, false, null); + Assertions.assertNull(kafkaProperties.get(MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY)); + + final String MAX_POLL_INTERVAL_MS = "100"; + final String HEARTBEAT_INTERVAL_MS = "30"; + + kafkaProperties.put(MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY, MAX_POLL_INTERVAL_MS); + kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS); + var kafkaConsumer = new KafkaConsumer(kafkaProperties); + var kafkaSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName); + var keysReceived = new ArrayList(); + + readNextNStreams(kafkaSource, keysReceived, 0, 1); + produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); + + var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY)) + .map(s->Integer.valueOf((String)s)).orElseThrow(); + Thread.sleep(pollIntervalMs*10); + kafkaSource.commitTrafficStream(keysReceived.get(0)); + log.info("finished committing traffic stream"); + readNextNStreams(kafkaSource, keysReceived, 1, 1); + Assertions.assertEquals(2, keysReceived.size()); + } + + @SneakyThrows + private static void readNextNStreams(KafkaProtobufConsumer kafkaSource, List keysReceived, + int from, int count) { + Assertions.assertEquals(from, keysReceived.size()); + for (int i=0; i{ + var tsk = ts.getKey(); + Assertions.assertFalse(keysReceived.contains(tsk)); + keysReceived.add(tsk); + }); + log.info("Read "+trafficStreams.size()+" traffic streams"); + i += trafficStreams.size(); + } + + } + //@Test @Tag("longTest") public void testTrafficCaptureSource() throws Exception { @@ -96,9 +159,9 @@ public void testTrafficCaptureSource() throws Exception { var kafkaConsumerProps = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), TEST_GROUP_CONSUMER_ID, false, null); - kafkaConsumerProps.setProperty("max.poll.interval.ms", "10000"); + kafkaConsumerProps.setProperty(MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY, "10000"); var kafkaConsumer = new KafkaConsumer(kafkaConsumerProps); - var kafkaTrafficCaptureSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName, null); + var kafkaTrafficCaptureSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName); var kafkaProducer = buildKafkaProducer(); var sendCompleteCount = new AtomicInteger(0); @@ -140,11 +203,11 @@ private long getSleepAmountMsForProducerRun(int i) { return 1*1000; } - private void produceKafkaRecord(String testTopicName, Producer kafkaProducer, int i, - AtomicInteger sendCompleteCount) { + private Future produceKafkaRecord(String testTopicName, Producer kafkaProducer, int i, + AtomicInteger sendCompleteCount) { var trafficStream = makeTestTrafficStream(Instant.now(), i); var record = new ProducerRecord(testTopicName, makeKey(i), trafficStream.toByteArray()); - var sendFuture = kafkaProducer.send(record, (metadata, exception) -> { + return kafkaProducer.send(record, (metadata, exception) -> { sendCompleteCount.incrementAndGet(); }); } From 43f529eea0f7fd1e466d870674f526a69506465d Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sat, 18 Nov 2023 18:16:35 -0500 Subject: [PATCH 2/9] Step 2 - implement a touch function that polls, then undoes by re-seeking so that things don't fall out while commits are pending. I still need to work on the case where commits might come back in after the consumer was kicked from the group, causing some gaps in re-replayed data (not violating at-least-once, but violating ordering). I also need to confirm that the kafka consumer won't bother trying to stay alive if timeouts are far into the future and no commits are pending. Signed-off-by: Greg Schohn --- .../src/test/resources/log4j2.properties | 2 +- .../replay/kafka/KafkaProtobufConsumer.java | 66 +++++++++++++++---- .../traffic/source/BlockingTrafficSource.java | 63 +++++++++++------- .../traffic/source/ITrafficCaptureSource.java | 5 +- .../KafkaRestartingTrafficReplayerTest.java | 9 ++- .../KafkaProtobufConsumerLongTermTest.java | 56 +++++++++++----- .../kafka/KafkaProtobufConsumerTest.java | 7 +- 7 files changed, 149 insertions(+), 59 deletions(-) diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties b/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties index 9c227b84f..702836711 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties @@ -6,6 +6,6 @@ appender.console.target = SYSTEM_ERR appender.console.layout.type = PatternLayout appender.console.layout.pattern = [%-5level] %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} [%t] %c{1} - %msg%equals{ ctx=%mdc}{ ctx=\{\}}{}%n -rootLogger.level = trace +rootLogger.level = debug rootLogger.appenderRefs = stderr rootLogger.appenderRef.stderr.ref = STDERR diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java index e2c5e9d7d..3b481b347 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java @@ -70,6 +70,8 @@ @Slf4j public class KafkaProtobufConsumer implements ISimpleTrafficCaptureSource { + public static final String MAX_POLL_INTERVAL_KEY = "max.poll.interval.ms"; + @ToString(callSuper = true) @EqualsAndHashCode(callSuper = true) private static class TrafficStreamKeyWithKafkaRecordId extends PojoTrafficStreamKey { @@ -128,17 +130,25 @@ Optional removeAndReturnNewHead(TrafficStreamKeyWithKafkaRecordId kafkaRec private final ConcurrentHashMap nextSetOfCommitsMap; private final Object offsetLifecycleLock = new Object(); private final String topic; + private final Duration keepAliveInterval; private final KafkaBehavioralPolicy behavioralPolicy; private final AtomicInteger trafficStreamsRead; + private Instant lastPollTimestamp; - public KafkaProtobufConsumer(Consumer kafkaConsumer, String topic) { - this(kafkaConsumer, topic, Clock.systemUTC(), new KafkaBehavioralPolicy()); + public KafkaProtobufConsumer(Consumer kafkaConsumer, String topic, Duration keepAliveInterval) { + this(kafkaConsumer, topic, keepAliveInterval, Clock.systemUTC(), new KafkaBehavioralPolicy()); } - public KafkaProtobufConsumer(Consumer kafkaConsumer, @NonNull String topic, - Clock clock, @NonNull KafkaBehavioralPolicy behavioralPolicy) { + public KafkaProtobufConsumer(Consumer kafkaConsumer, + @NonNull String topic, + Duration keepAliveInterval, + Clock clock, + @NonNull KafkaBehavioralPolicy behavioralPolicy) + { this.kafkaConsumer = kafkaConsumer; this.topic = topic; + this.keepAliveInterval = keepAliveInterval; + log.error("keepAliveInterval="+keepAliveInterval); this.behavioralPolicy = behavioralPolicy; kafkaConsumer.subscribe(Collections.singleton(topic)); trafficStreamsRead = new AtomicInteger(); @@ -158,7 +168,14 @@ public static KafkaProtobufConsumer buildKafkaConsumer(@NonNull String brokers, throws IOException { var kafkaProps = buildKafkaProperties(brokers, groupId, enableMSKAuth, propertyFilePath); - return new KafkaProtobufConsumer(new KafkaConsumer<>(kafkaProps), topic, clock, behavioralPolicy); + var pollPeriod = Duration.ofMillis(Integer.valueOf((String)kafkaProps.get(MAX_POLL_INTERVAL_KEY))); + var keepAlivePeriod = getKeepAlivePeriodFromPollPeriod(pollPeriod); + return new KafkaProtobufConsumer(new KafkaConsumer<>(kafkaProps), topic, keepAlivePeriod, + clock, behavioralPolicy); + } + + private static Duration getKeepAlivePeriodFromPollPeriod(Duration pollPeriod) { + return pollPeriod.dividedBy(2); } public static Properties buildKafkaProperties(@NonNull String brokers, @@ -190,14 +207,40 @@ public static Properties buildKafkaProperties(@NonNull String brokers, return kafkaProps; } -// @Override -// public Optional maximumKeepAliveInterval() { -// return Optional.of(Duration.); -// } + @Override + public void touch() { + log.error("TOUCH CALLED"); + try { + var records = kafkaConsumer.poll(Duration.ZERO); + log.atError().setMessage(()->"Polled "+records.count()+" records to keep the consumer alive").log(); + records.forEach(r -> { + try { + var tp = new TopicPartition(r.topic(), r.partition()); + log.atError().setMessage(()->"Resetting "+tp+" to offset="+r.offset()).log(); + kafkaConsumer.seek(tp, r.offset()); + } catch (IllegalStateException e) { + log.atWarn().setCause(e).setMessage(() -> "Caught exception while seeking. " + + "Ignoring so that other records can have their seeks readjusted.").log(); + } + }); + } catch (RuntimeException e) { + log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + + "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); + } + lastPollTimestamp = clock.instant(); + } + /** + * If messages are outstanding, we need to keep the connection alive, otherwise, there's no + * reason to. It's OK to fall out of the group and rejoin once ready. + * @return + */ @Override - public Optional touch() { - return Optional.of(Instant.MAX); + public Optional getNextRequiredTouch() { + synchronized (offsetLifecycleLock) { + return partitionToOffsetLifecycleTrackerMap.isEmpty() ? Optional.empty() : + Optional.of(lastPollTimestamp.plus(keepAliveInterval)); + } } @Override @@ -268,6 +311,7 @@ private void safeCommitWithRetry() { private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { try { + lastPollTimestamp = clock.instant(); var records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT); log.atLevel(records.isEmpty()? Level.TRACE:Level.INFO) .setMessage(()->"Kafka consumer poll has fetched "+records.count()+" records").log(); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java index a22ad2549..ab36d383f 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java @@ -15,6 +15,7 @@ import java.util.StringJoiner; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; /** @@ -40,9 +41,9 @@ public class BlockingTrafficSource implements ITrafficCaptureSource, BufferedFlo * Limit the number of readers to one at a time and only if we haven't yet maxed out our time buffer */ private final Semaphore readGate; - private final Duration bufferTimeWindow; + public BlockingTrafficSource(ISimpleTrafficCaptureSource underlying, Duration bufferTimeWindow) { this.underlyingSource = underlying; @@ -89,27 +90,9 @@ public Duration getBufferTimeWindow() { @Override public CompletableFuture> readNextTrafficStreamChunk() { - var trafficStreamListFuture = - CompletableFuture.supplyAsync(() -> { - if (stopReadingAtRef.get().equals(Instant.EPOCH)) { return null; } - while (stopReadingAtRef.get().isBefore(lastTimestampSecondsRef.get())) { - try { - log.atInfo().setMessage( - "blocking until signaled to read the next chunk last={} stop={}") - .addArgument(lastTimestampSecondsRef.get()) - .addArgument(stopReadingAtRef.get()) - .log(); - readGate.acquire(); - } catch (InterruptedException e) { - log.atWarn().setCause(e).log("Interrupted while waiting to read more data"); - Thread.currentThread().interrupt(); - break; - } - } - return null; - }, - task -> new Thread(task).start()) - .thenCompose(v->underlyingSource.readNextTrafficStreamChunk()); + var trafficStreamListFuture = CompletableFuture + .supplyAsync(this::blockIfNeeded, task -> new Thread(task).start()) + .thenCompose(v->underlyingSource.readNextTrafficStreamChunk()); return trafficStreamListFuture.whenComplete((v,t)->{ if (t != null) { return; @@ -126,6 +109,42 @@ public Duration getBufferTimeWindow() { }); } + private Void blockIfNeeded() { + if (stopReadingAtRef.get().equals(Instant.EPOCH)) { return null; } + log.atInfo().setMessage(()->"stopReadingAtRef="+stopReadingAtRef+ + " lastTimestampSecondsRef="+lastTimestampSecondsRef).log(); + while (stopReadingAtRef.get().isBefore(lastTimestampSecondsRef.get())) { + try { + log.atInfo().setMessage( + "blocking until signaled to read the next chunk last={} stop={}") + .addArgument(lastTimestampSecondsRef.get()) + .addArgument(stopReadingAtRef.get()) + .log(); + var nextTouchOp = underlyingSource.getNextRequiredTouch(); + if (nextTouchOp.isEmpty()) { + readGate.acquire(); + } else { + var nextInstant = nextTouchOp.get(); + final var nowTime = Instant.now(); + var waitIntervalMs = Duration.between(nowTime, nextInstant).toMillis(); + log.error("Next touch at " + nextInstant + " ... in " + waitIntervalMs + "ms (now="+nowTime+")"); + if (waitIntervalMs <= 0) { + underlyingSource.touch(); + } else { + // if this doesn't succeed, we'll loop around & likely do a touch, then loop around again. + // if it DOES succeed, we'll loop around and make sure that there's not another reason to stop + readGate.tryAcquire(waitIntervalMs, TimeUnit.MILLISECONDS); + } + } + } catch (InterruptedException e) { + log.atWarn().setCause(e).log("Interrupted while waiting to read more data"); + Thread.currentThread().interrupt(); + break; + } + } + return null; + } + @Override public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) throws IOException { underlyingSource.commitTrafficStream(trafficStreamKey); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java index b0601e2bc..4e637cfcf 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java @@ -25,7 +25,6 @@ default void close() throws IOException {} * active. Empty indicates that touch() does not need to be called to keep the * source active. */ - default Optional touch() { - return Optional.empty(); - } + default void touch() {} + default Optional getNextRequiredTouch() { return Optional.empty(); } } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java index a9398bb81..bff5d2c9c 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java @@ -50,6 +50,7 @@ public class KafkaRestartingTrafficReplayerTest { public static final int PRODUCER_SLEEP_INTERVAL_MS = 100; public static final Duration MAX_WAIT_TIME_FOR_TOPIC = Duration.ofMillis(PRODUCER_SLEEP_INTERVAL_MS*2); + public static final long DEFAULT_POLL_INTERVAL_MS = 5000; @Container // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility @@ -98,7 +99,8 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(streamAndConsumer.numHttpTransactions, httpServer.localhostEndpoint(), new CounterLimitedReceiverFactory(), () -> new SentinelSensingTrafficSource( - new KafkaProtobufConsumer(buildKafkaConsumer(), TEST_TOPIC_NAME))); + new KafkaProtobufConsumer(buildKafkaConsumer(), TEST_TOPIC_NAME, + Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS)))); log.error("done"); } @@ -106,7 +108,7 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { private KafkaConsumer buildKafkaConsumer() { var kafkaConsumerProps = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), TEST_GROUP_CONSUMER_ID, false, null); - kafkaConsumerProps.setProperty("max.poll.interval.ms", "5000"); + kafkaConsumerProps.setProperty("max.poll.interval.ms", DEFAULT_POLL_INTERVAL_MS+""); var kafkaConsumer = new KafkaConsumer(kafkaConsumerProps); log.atInfo().setMessage(()->"Just built KafkaConsumer="+kafkaConsumer).log(); return kafkaConsumer; @@ -174,7 +176,8 @@ Producer buildKafkaProducer() { throw Lombok.sneakyThrow(e); } }); - return () -> new KafkaProtobufConsumer(kafkaConsumer, TEST_TOPIC_NAME); + return () -> new KafkaProtobufConsumer(kafkaConsumer, TEST_TOPIC_NAME, + Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS)); } @SneakyThrows diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java index c057ca314..738ee7830 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java @@ -15,6 +15,7 @@ import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.trafficcapture.protos.ReadObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; @@ -24,6 +25,7 @@ import org.testcontainers.utility.DockerImageName; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.time.Instant; import java.util.ArrayList; import java.util.List; @@ -34,7 +36,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.IntStream; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -49,7 +50,6 @@ public class KafkaProtobufConsumerLongTermTest { private static final String FAKE_READ_PACKET_DATA = "Fake pa"; public static final int PRODUCER_SLEEP_INTERVAL_MS = 100; - public static final String MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY = "max.poll.interval.ms"; public static final String HEARTBEAT_INTERVAL_MS_KEY = "heartbeat.interval.ms"; @Container @@ -105,6 +105,9 @@ private String getConnectionId(int i) { public void testTimeoutsDontOccurForSlowPolls() throws Exception { String testTopicName = "TEST_TOPIC"; + log.info("Starting test"); + log.error("Starting test"); + var kafkaProducer = buildKafkaProducer(); var sendCompleteCount = new AtomicInteger(0); produceKafkaRecord(testTopicName, kafkaProducer, 0, sendCompleteCount).get(); @@ -112,31 +115,48 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { var kafkaProperties = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), TEST_GROUP_CONSUMER_ID, false, null); - Assertions.assertNull(kafkaProperties.get(MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY)); + Assertions.assertNull(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)); - final String MAX_POLL_INTERVAL_MS = "100"; - final String HEARTBEAT_INTERVAL_MS = "30"; + final long MAX_POLL_INTERVAL_MS = 1000; + final long HEARTBEAT_INTERVAL_MS = 300; - kafkaProperties.put(MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY, MAX_POLL_INTERVAL_MS); - kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS); + kafkaProperties.put(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY, MAX_POLL_INTERVAL_MS+""); + kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS+""); var kafkaConsumer = new KafkaConsumer(kafkaProperties); - var kafkaSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName); + var trafficSource = new BlockingTrafficSource( + new KafkaProtobufConsumer(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)), + Duration.ZERO); var keysReceived = new ArrayList(); - readNextNStreams(kafkaSource, keysReceived, 0, 1); + readNextNStreams(trafficSource, keysReceived, 0, 1); + trafficSource.stopReadsPast(Instant.EPOCH.plus(Duration.ofDays(1))); produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); - var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(MAX_POLL_INTERVAL_BEFORE_EVICTION_KEY)) + var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)) .map(s->Integer.valueOf((String)s)).orElseThrow(); - Thread.sleep(pollIntervalMs*10); - kafkaSource.commitTrafficStream(keysReceived.get(0)); + var executor = Executors.newSingleThreadScheduledExecutor(); + executor.schedule(()-> { + try { + var k = keysReceived.get(0); + log.warn("Calling commit traffic stream for "+k); + trafficSource.commitTrafficStream(k); + trafficSource.stopReadsPast(Instant.MAX); + log.warn("Stop reads past infinity"); + Thread.sleep(1000); + produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); + } catch (Exception e) { + Lombok.sneakyThrow(e); + } + }, + pollIntervalMs, TimeUnit.MILLISECONDS); + log.info("finished committing traffic stream"); - readNextNStreams(kafkaSource, keysReceived, 1, 1); - Assertions.assertEquals(2, keysReceived.size()); + readNextNStreams(trafficSource, keysReceived, 1, 2); + Assertions.assertEquals(3, keysReceived.size()); } @SneakyThrows - private static void readNextNStreams(KafkaProtobufConsumer kafkaSource, List keysReceived, + private static void readNextNStreams(BlockingTrafficSource kafkaSource, List keysReceived, int from, int count) { Assertions.assertEquals(from, keysReceived.size()); for (int i=0; i(kafkaConsumerProps); - var kafkaTrafficCaptureSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName); + var kafkaTrafficCaptureSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName, + Duration.ofMillis(MAX_POLL_MS)); var kafkaProducer = buildKafkaProducer(); var sendCompleteCount = new AtomicInteger(0); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerTest.java index 8ae5c46d3..073cab88c 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerTest.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Optional; import java.util.Properties; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; @@ -40,7 +41,8 @@ class KafkaProtobufConsumerTest { public void testSupplyTrafficFromSource() { int numTrafficStreams = 10; MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - KafkaProtobufConsumer protobufConsumer = new KafkaProtobufConsumer(mockConsumer, TEST_TOPIC_NAME); + KafkaProtobufConsumer protobufConsumer = new KafkaProtobufConsumer(mockConsumer, TEST_TOPIC_NAME, + Duration.ofHours(1)); initializeMockConsumerTopic(mockConsumer); List substreamCounts = new ArrayList<>(); @@ -80,7 +82,8 @@ public void testSupplyTrafficFromSource() { public void testSupplyTrafficWithUnformattedMessages() { int numTrafficStreams = 10; MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - KafkaProtobufConsumer protobufConsumer = new KafkaProtobufConsumer(mockConsumer, TEST_TOPIC_NAME); + KafkaProtobufConsumer protobufConsumer = new KafkaProtobufConsumer(mockConsumer, TEST_TOPIC_NAME, + Duration.ofHours(1)); initializeMockConsumerTopic(mockConsumer); List substreamCounts = new ArrayList<>(); From 1eb596877f0a94eb581b15af429a87dfb82d78a5 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sun, 19 Nov 2023 08:16:25 -0500 Subject: [PATCH 3/9] Refactor some Kafka tests to make it easier to extract common code to setup shared sub-scenarios. Signed-off-by: Greg Schohn --- .../replay/kafka/KafkaKeepAliveTests.java | 186 ++++++++++++++++++ .../KafkaProtobufConsumerLongTermTest.java | 145 +------------- .../replay/kafka/KafkaTestUtils.java | 89 +++++++++ 3 files changed, 282 insertions(+), 138 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java create mode 100644 TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java new file mode 100644 index 000000000..d78b9c1ed --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java @@ -0,0 +1,186 @@ +package org.opensearch.migrations.replay.kafka; + +import lombok.Lombok; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Producer; +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +@Slf4j +@Testcontainers(disabledWithoutDocker = true) +@Tag("requiresDocker") +public class KafkaKeepAliveTests { + public static final String TEST_GROUP_CONSUMER_ID = "TEST_GROUP_CONSUMER_ID"; + public static final String HEARTBEAT_INTERVAL_MS_KEY = "heartbeat.interval.ms"; + public static final long MAX_POLL_INTERVAL_MS = 1000; + public static final long HEARTBEAT_INTERVAL_MS = 300; + public static final String testTopicName = "TEST_TOPIC"; + + Producer kafkaProducer; + AtomicInteger sendCompleteCount; + Properties kafkaProperties; + BlockingTrafficSource trafficSource; + ArrayList keysReceived; + + @Container + // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility + private KafkaContainer embeddedKafkaBroker = + new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); + + @BeforeEach + private void setupTestCase() throws Exception { + kafkaProducer = KafkaTestUtils.buildKafkaProducer(embeddedKafkaBroker.getBootstrapServers()); + this.sendCompleteCount = new AtomicInteger(0); + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 0, sendCompleteCount).get(); + Assertions.assertEquals(1, sendCompleteCount.get()); + + this.kafkaProperties = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), + TEST_GROUP_CONSUMER_ID, false, null); + Assertions.assertNull(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)); + + kafkaProperties.put(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY, MAX_POLL_INTERVAL_MS+""); + kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS+""); + var kafkaConsumer = new KafkaConsumer(kafkaProperties); + this.trafficSource = new BlockingTrafficSource( + new KafkaProtobufConsumer(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)), + Duration.ZERO); + this.keysReceived = new ArrayList(); + + readNextNStreams(trafficSource, keysReceived, 0, 1); + trafficSource.stopReadsPast(Instant.EPOCH.plus(Duration.ofDays(1))); + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); + } + + + @Test + @Tag("longTest") + public void testTimeoutsDontOccurForSlowPolls() throws Exception { + var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)) + .map(s->Integer.valueOf((String)s)).orElseThrow(); + var executor = Executors.newSingleThreadScheduledExecutor(); + executor.schedule(()-> { + try { + var k = keysReceived.get(0); + log.warn("Calling commit traffic stream for "+k); + trafficSource.commitTrafficStream(k); + log.info("finished committing traffic stream"); + trafficSource.stopReadsPast(Instant.MAX); + log.warn("Stop reads past infinity"); + // this is a way to signal back to the main thread that this thread is done + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); + } catch (Exception e) { + Lombok.sneakyThrow(e); + } + }, + pollIntervalMs, TimeUnit.MILLISECONDS); + + // wait for 2 messages so that they include the last one produced by the async schedule call previously + readNextNStreams(trafficSource, keysReceived, 1, 2); + Assertions.assertEquals(3, keysReceived.size()); + // At this point, we've read all (3) messages produced , committed the first one + // (all the way through to Kafka), and no commits are in-flight yet for the last two messages. + } + + + @Test + @Tag("longTest") + public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Exception { + var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)) + .map(s->Integer.valueOf((String)s)).orElseThrow(); + var executor = Executors.newSingleThreadScheduledExecutor(); + executor.schedule(()-> { + try { + var k = keysReceived.get(0); + log.warn("Calling commit traffic stream for "+k); + trafficSource.commitTrafficStream(k); + log.info("finished committing traffic stream"); + trafficSource.stopReadsPast(Instant.MAX); + log.warn("Stop reads past infinity"); + // this is a way to signal back to the main thread that this thread is done + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); + } catch (Exception e) { + Lombok.sneakyThrow(e); + } + }, + pollIntervalMs, TimeUnit.MILLISECONDS); + + // wait for 2 messages so that they include the last one produced by the async schedule call previously + readNextNStreams(trafficSource, keysReceived, 1, 2); + Assertions.assertEquals(3, keysReceived.size()); + // At this point, we've read all (3) messages produced , committed the first one + // (all the way through to Kafka), and no commits are in-flight yet for the last two messages. + + // the first message was committed even though we waited to pull the next message only because of touches + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 3, sendCompleteCount); + readNextNStreams(trafficSource, keysReceived,2, 1 ); + // + + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 4, sendCompleteCount); + + // wait long enough to call the BlockingTrafficSource so that the kafka consumer will fall out + // and make sure that we have only committed the one message within the delayed scheduler above + // (we should have definitely hit that since readNextNStreams() will block + Thread.sleep(2* MAX_POLL_INTERVAL_MS); + readNextNStreams(trafficSource, keysReceived, 3, 1); + Assertions.assertEquals(4, keysReceived.size()); + + + + + + // the first message was committed even though we waited to pull the next message only because of touches + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 3, sendCompleteCount); + readNextNStreams(trafficSource, keysReceived,2, 1 ); + // + + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 4, sendCompleteCount); + + // wait long enough to call the BlockingTrafficSource so that the kafka consumer will fall out + // and make sure that we have only committed the one message within the delayed scheduler above + // (we should have definitely hit that since readNextNStreams() will block + Thread.sleep(2* MAX_POLL_INTERVAL_MS); + readNextNStreams(trafficSource, keysReceived, 3, 1); + Assertions.assertEquals(4, keysReceived.size()); + + } + + @SneakyThrows + private static void readNextNStreams(BlockingTrafficSource kafkaSource, List keysReceived, + int from, int count) { + Assertions.assertEquals(from, keysReceived.size()); + for (int i=0; i{ + var tsk = ts.getKey(); + log.atInfo().setMessage(()->"checking for "+tsk).log(); + Assertions.assertFalse(keysReceived.contains(tsk)); + keysReceived.add(tsk); + }); + log.info("Read "+trafficStreams.size()+" traffic streams"); + i += trafficStreams.size(); + } + } +} diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java index 738ee7830..e87dbab93 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java @@ -19,11 +19,13 @@ import org.opensearch.migrations.trafficcapture.protos.ReadObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; import org.testcontainers.utility.DockerImageName; +import java.io.IOException; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; @@ -31,6 +33,7 @@ import java.util.List; import java.util.Optional; import java.util.Properties; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -42,138 +45,18 @@ @Tag("requiresDocker") public class KafkaProtobufConsumerLongTermTest { - public static final String TEST_GROUP_CONSUMER_ID = "TEST_GROUP_CONSUMER_ID"; - public static final String TEST_GROUP_PRODUCER_ID = "TEST_GROUP_PRODUCER_ID"; public static final int TEST_RECORD_COUNT = 10; - public static final String TEST_NODE_ID = "TestNodeId"; - public static final String TEST_TRAFFIC_STREAM_ID_STRING = "TEST_TRAFFIC_STREAM_ID_STRING"; - private static final String FAKE_READ_PACKET_DATA = "Fake pa"; + public static final String TEST_GROUP_CONSUMER_ID = "TEST_GROUP_CONSUMER_ID"; public static final int PRODUCER_SLEEP_INTERVAL_MS = 100; - public static final String HEARTBEAT_INTERVAL_MS_KEY = "heartbeat.interval.ms"; - @Container // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility private KafkaContainer embeddedKafkaBroker = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); - Producer buildKafkaProducer() { - var kafkaProps = new Properties(); - kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); - kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - // Property details: https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html#delivery-timeout-ms - kafkaProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, 10000); - kafkaProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000); - kafkaProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); - kafkaProps.put(ProducerConfig.CLIENT_ID_CONFIG, TEST_GROUP_PRODUCER_ID); - kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, embeddedKafkaBroker.getBootstrapServers()); - try { - return new KafkaProducer(kafkaProps); - } catch (Exception e) { - log.atError().setCause(e).log(); - System.exit(1); - throw e; - } - } - - TrafficStream makeTestTrafficStream(Instant t, int i) { - var timestamp = Timestamp.newBuilder() - .setSeconds(t.getEpochSecond()) - .setNanos(t.getNano()) - .build(); - var tsb = TrafficStream.newBuilder() - .setNumber(i); - // TODO - add something for setNumberOfThisLastChunk. There's no point in doing that now though - // because the code doesn't make any distinction between the very last one and the previous ones - return tsb.setNodeId(TEST_NODE_ID) - .setConnectionId(getConnectionId(i)) - .addSubStream(TrafficObservation.newBuilder().setTs(timestamp) - .setRead(ReadObservation.newBuilder() - .setData(ByteString.copyFrom(FAKE_READ_PACKET_DATA.getBytes(StandardCharsets.UTF_8))) - .build()) - .build()).build(); - - } - - private String getConnectionId(int i) { - return TEST_TRAFFIC_STREAM_ID_STRING + "_" + i; - } - @Test @Tag("longTest") - public void testTimeoutsDontOccurForSlowPolls() throws Exception { - String testTopicName = "TEST_TOPIC"; - - log.info("Starting test"); - log.error("Starting test"); - - var kafkaProducer = buildKafkaProducer(); - var sendCompleteCount = new AtomicInteger(0); - produceKafkaRecord(testTopicName, kafkaProducer, 0, sendCompleteCount).get(); - Assertions.assertEquals(1, sendCompleteCount.get()); - - var kafkaProperties = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), - TEST_GROUP_CONSUMER_ID, false, null); - Assertions.assertNull(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)); - - final long MAX_POLL_INTERVAL_MS = 1000; - final long HEARTBEAT_INTERVAL_MS = 300; - - kafkaProperties.put(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY, MAX_POLL_INTERVAL_MS+""); - kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS+""); - var kafkaConsumer = new KafkaConsumer(kafkaProperties); - var trafficSource = new BlockingTrafficSource( - new KafkaProtobufConsumer(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)), - Duration.ZERO); - var keysReceived = new ArrayList(); - - readNextNStreams(trafficSource, keysReceived, 0, 1); - trafficSource.stopReadsPast(Instant.EPOCH.plus(Duration.ofDays(1))); - produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); - - var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)) - .map(s->Integer.valueOf((String)s)).orElseThrow(); - var executor = Executors.newSingleThreadScheduledExecutor(); - executor.schedule(()-> { - try { - var k = keysReceived.get(0); - log.warn("Calling commit traffic stream for "+k); - trafficSource.commitTrafficStream(k); - trafficSource.stopReadsPast(Instant.MAX); - log.warn("Stop reads past infinity"); - Thread.sleep(1000); - produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); - } catch (Exception e) { - Lombok.sneakyThrow(e); - } - }, - pollIntervalMs, TimeUnit.MILLISECONDS); - - log.info("finished committing traffic stream"); - readNextNStreams(trafficSource, keysReceived, 1, 2); - Assertions.assertEquals(3, keysReceived.size()); - } - - @SneakyThrows - private static void readNextNStreams(BlockingTrafficSource kafkaSource, List keysReceived, - int from, int count) { - Assertions.assertEquals(from, keysReceived.size()); - for (int i=0; i{ - var tsk = ts.getKey(); - Assertions.assertFalse(keysReceived.contains(tsk)); - keysReceived.add(tsk); - }); - log.info("Read "+trafficStreams.size()+" traffic streams"); - i += trafficStreams.size(); - } - - } - - //@Test - @Tag("longTest") public void testTrafficCaptureSource() throws Exception { String testTopicName = "TEST_TOPIC"; @@ -185,7 +68,7 @@ public void testTrafficCaptureSource() throws Exception { var kafkaTrafficCaptureSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_MS)); - var kafkaProducer = buildKafkaProducer(); + var kafkaProducer = KafkaTestUtils.buildKafkaProducer(embeddedKafkaBroker.getBootstrapServers()); var sendCompleteCount = new AtomicInteger(0); var scheduledIterationsCount = new AtomicInteger(0); var executor = Executors.newSingleThreadScheduledExecutor(); @@ -194,7 +77,7 @@ public void testTrafficCaptureSource() throws Exception { if (i >= TEST_RECORD_COUNT) { executor.shutdown(); } else { - produceKafkaRecord(testTopicName, kafkaProducer, i, sendCompleteCount); + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, i, sendCompleteCount); } }, 0, PRODUCER_SLEEP_INTERVAL_MS, TimeUnit.MILLISECONDS); @@ -203,7 +86,7 @@ public void testTrafficCaptureSource() throws Exception { var nextChunkFuture = kafkaTrafficCaptureSource.readNextTrafficStreamChunk(); var recordsList = nextChunkFuture.get((2+ TEST_RECORD_COUNT)*PRODUCER_SLEEP_INTERVAL_MS, TimeUnit.MILLISECONDS); for (int j=0; j kafkaProducer, int i, - AtomicInteger sendCompleteCount) { - var trafficStream = makeTestTrafficStream(Instant.now(), i); - var record = new ProducerRecord(testTopicName, makeKey(i), trafficStream.toByteArray()); - return kafkaProducer.send(record, (metadata, exception) -> { - sendCompleteCount.incrementAndGet(); - }); - } - - @NotNull - private static String makeKey(int i) { - return "KEY_" + i; - } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java new file mode 100644 index 000000000..329bf90ac --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java @@ -0,0 +1,89 @@ +package org.opensearch.migrations.replay.kafka; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.Tag; +import org.opensearch.migrations.trafficcapture.protos.ReadObservation; +import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; +import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; + +@Slf4j +public class KafkaTestUtils { + + public static final String TEST_GROUP_PRODUCER_ID = "TEST_GROUP_PRODUCER_ID"; + private static final String FAKE_READ_PACKET_DATA = "Fake pa"; + public static final String TEST_NODE_ID = "TestNodeId"; + public static final String TEST_TRAFFIC_STREAM_ID_STRING = "TEST_TRAFFIC_STREAM_ID_STRING"; + + static Producer buildKafkaProducer(String bootstrapServers) { + var kafkaProps = new Properties(); + kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); + kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + // Property details: https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html#delivery-timeout-ms + kafkaProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, 10000); + kafkaProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000); + kafkaProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); + kafkaProps.put(ProducerConfig.CLIENT_ID_CONFIG, TEST_GROUP_PRODUCER_ID); + kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + try { + return new KafkaProducer(kafkaProps); + } catch (Exception e) { + log.atError().setCause(e).log(); + System.exit(1); + throw e; + } + } + + static String getConnectionId(int i) { + return TEST_TRAFFIC_STREAM_ID_STRING + "_" + i; + } + + static TrafficStream makeTestTrafficStream(Instant t, int i) { + var timestamp = Timestamp.newBuilder() + .setSeconds(t.getEpochSecond()) + .setNanos(t.getNano()) + .build(); + var tsb = TrafficStream.newBuilder() + .setNumber(i); + // TODO - add something for setNumberOfThisLastChunk. There's no point in doing that now though + // because the code doesn't make any distinction between the very last one and the previous ones + return tsb.setNodeId(TEST_NODE_ID) + .setConnectionId(getConnectionId(i)) + .addSubStream(TrafficObservation.newBuilder().setTs(timestamp) + .setRead(ReadObservation.newBuilder() + .setData(ByteString.copyFrom(FAKE_READ_PACKET_DATA.getBytes(StandardCharsets.UTF_8))) + .build()) + .build()).build(); + + } + + static Future produceKafkaRecord(String testTopicName, Producer kafkaProducer, + int i, AtomicInteger sendCompleteCount) { + var trafficStream = KafkaTestUtils.makeTestTrafficStream(Instant.now(), i); + var record = new ProducerRecord(testTopicName, makeKey(i), trafficStream.toByteArray()); + return kafkaProducer.send(record, (metadata, exception) -> { + sendCompleteCount.incrementAndGet(); + }); + } + + @NotNull + private static String makeKey(int i) { + return "KEY_" + i; + } +} From e5262000d5f655317c82312798f9fb45352b0796 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Sun, 19 Nov 2023 16:34:00 -0500 Subject: [PATCH 4/9] Failed attempts to make sure that commits do NOT commit messages that may be getting processed simultaneously elsewhere. Signed-off-by: Greg Schohn --- .../replay/kafka/KafkaProtobufConsumer.java | 34 ++++- .../replay/kafka/KafkaKeepAliveTests.java | 126 +++++++++--------- .../replay/kafka/KafkaTestUtils.java | 3 +- .../src/test/resources/log4j2.properties | 4 + 4 files changed, 95 insertions(+), 72 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java index 3b481b347..204fe9e26 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java @@ -77,9 +77,11 @@ public class KafkaProtobufConsumer implements ISimpleTrafficCaptureSource { private static class TrafficStreamKeyWithKafkaRecordId extends PojoTrafficStreamKey { private final int partition; private final long offset; + private final int generation; - TrafficStreamKeyWithKafkaRecordId(TrafficStream trafficStream, int partition, long offset) { + TrafficStreamKeyWithKafkaRecordId(int generation, TrafficStream trafficStream, int partition, long offset) { super(trafficStream); + this.generation = generation; this.partition = partition; this.offset = offset; } @@ -126,14 +128,16 @@ Optional removeAndReturnNewHead(TrafficStreamKeyWithKafkaRecordId kafkaRec private final Clock clock; private final Consumer kafkaConsumer; - private final ConcurrentHashMap partitionToOffsetLifecycleTrackerMap; - private final ConcurrentHashMap nextSetOfCommitsMap; + final ConcurrentHashMap partitionToOffsetLifecycleTrackerMap; + // loosening visibility so that a unit test can read this + final ConcurrentHashMap nextSetOfCommitsMap; private final Object offsetLifecycleLock = new Object(); private final String topic; private final Duration keepAliveInterval; private final KafkaBehavioralPolicy behavioralPolicy; private final AtomicInteger trafficStreamsRead; private Instant lastPollTimestamp; + private int consumerConnectionGeneration; public KafkaProtobufConsumer(Consumer kafkaConsumer, String topic, Duration keepAliveInterval) { this(kafkaConsumer, topic, keepAliveInterval, Clock.systemUTC(), new KafkaBehavioralPolicy()); @@ -226,7 +230,9 @@ public void touch() { } catch (RuntimeException e) { log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); + resetCommitInfo(); } + safeCommitWithRetry(); lastPollTimestamp = clock.instant(); } @@ -264,7 +270,8 @@ public List readNextTrafficStreamSynchronously() { .setAttribute(MetricsAttributeKey.TOPIC_NAME, this.topic) .setAttribute(MetricsAttributeKey.SIZE_IN_BYTES, ts.getSerializedSize()).emit(); addOffset(kafkaRecord.partition(), kafkaRecord.offset()); - var key = new TrafficStreamKeyWithKafkaRecordId(ts, kafkaRecord.partition(), kafkaRecord.offset()); + var key = new TrafficStreamKeyWithKafkaRecordId(consumerConnectionGeneration, ts, + kafkaRecord.partition(), kafkaRecord.offset()); log.atTrace().setMessage(()->"Parsed traffic stream #{}: {} {}") .addArgument(trafficStreamsRead.incrementAndGet()) .addArgument(key) @@ -296,6 +303,7 @@ private void safeCommitWithRetry() { kafkaConsumer.commitSync(nextSetOfCommitsMap); log.atDebug().setMessage(() -> "Done committing " + nextSetOfCommitsMap).log(); } + nextSetOfCommitsMap.clear(); } catch (RuntimeException e) { log.atWarn().setCause(e) .setMessage(() -> "Error while committing. Purging all commit points since another consumer " + @@ -303,12 +311,18 @@ private void safeCommitWithRetry() { "discarded: " + nextSetOfCommitsMap.entrySet().stream() .map(kvp -> kvp.getKey() + "->" + kvp.getValue()).collect(Collectors.joining(","))) .log(); - } finally { - nextSetOfCommitsMap.clear(); + resetCommitInfo(); } } } + private void resetCommitInfo() { + ++consumerConnectionGeneration; + nextSetOfCommitsMap.clear(); + partitionToOffsetLifecycleTrackerMap.clear(); + + } + private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { try { lastPollTimestamp = clock.instant(); @@ -323,6 +337,7 @@ private ConsumerRecords safePollWithSwallowedRuntimeExceptions() } catch (RuntimeException e) { log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); + resetCommitInfo(); return new ConsumerRecords<>(Collections.emptyMap()); } } @@ -342,6 +357,13 @@ public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { " but received "+trafficStreamKey+" (of type="+trafficStreamKey.getClass()+")"); } var kafkaTsk = (TrafficStreamKeyWithKafkaRecordId) trafficStreamKey; + if (kafkaTsk.generation != consumerConnectionGeneration) { + log.atWarn().setMessage(()->"trafficKey's generation (" + kafkaTsk.generation + ") is not current (" + + consumerConnectionGeneration + "). Dropping this commit request since the record would have " + + "been or will be handled again by a current consumer within this process or another. Full key=" + + kafkaTsk).log(); + return; + } var p = kafkaTsk.partition; Optional newHeadValue; synchronized (offsetLifecycleLock) { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java index d78b9c1ed..5d21d56ab 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java @@ -1,5 +1,6 @@ package org.opensearch.migrations.replay.kafka; +import com.beust.ah.A; import lombok.Lombok; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -28,6 +29,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -49,7 +51,14 @@ public class KafkaKeepAliveTests { // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility private KafkaContainer embeddedKafkaBroker = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); - + private KafkaProtobufConsumer kafkaSource; + + /** + * Setup the test case where we've produced and received 1 message, but have not yet committed it. + * Another message is in the process of being produced. + * The BlockingTrafficSource is blocked on everything after a point before the beginning of the test. + * @throws Exception + */ @BeforeEach private void setupTestCase() throws Exception { kafkaProducer = KafkaTestUtils.buildKafkaProducer(embeddedKafkaBroker.getBootstrapServers()); @@ -63,17 +72,16 @@ private void setupTestCase() throws Exception { kafkaProperties.put(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY, MAX_POLL_INTERVAL_MS+""); kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS+""); + kafkaProperties.put("max.poll.records", 1); var kafkaConsumer = new KafkaConsumer(kafkaProperties); - this.trafficSource = new BlockingTrafficSource( - new KafkaProtobufConsumer(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)), - Duration.ZERO); + this.kafkaSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)); + this.trafficSource = new BlockingTrafficSource(kafkaSource, Duration.ZERO); this.keysReceived = new ArrayList(); readNextNStreams(trafficSource, keysReceived, 0, 1); - trafficSource.stopReadsPast(Instant.EPOCH.plus(Duration.ofDays(1))); + stopReadsPast(1); KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); - } - + } @Test @Tag("longTest") @@ -87,7 +95,7 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { log.warn("Calling commit traffic stream for "+k); trafficSource.commitTrafficStream(k); log.info("finished committing traffic stream"); - trafficSource.stopReadsPast(Instant.MAX); + stopReadsPast(2); log.warn("Stop reads past infinity"); // this is a way to signal back to the main thread that this thread is done KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); @@ -104,69 +112,57 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { // (all the way through to Kafka), and no commits are in-flight yet for the last two messages. } - @Test @Tag("longTest") public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Exception { - var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)) - .map(s->Integer.valueOf((String)s)).orElseThrow(); - var executor = Executors.newSingleThreadScheduledExecutor(); - executor.schedule(()-> { - try { - var k = keysReceived.get(0); - log.warn("Calling commit traffic stream for "+k); - trafficSource.commitTrafficStream(k); - log.info("finished committing traffic stream"); - trafficSource.stopReadsPast(Instant.MAX); - log.warn("Stop reads past infinity"); - // this is a way to signal back to the main thread that this thread is done - KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); - } catch (Exception e) { - Lombok.sneakyThrow(e); - } - }, - pollIntervalMs, TimeUnit.MILLISECONDS); - - // wait for 2 messages so that they include the last one produced by the async schedule call previously - readNextNStreams(trafficSource, keysReceived, 1, 2); - Assertions.assertEquals(3, keysReceived.size()); - // At this point, we've read all (3) messages produced , committed the first one - // (all the way through to Kafka), and no commits are in-flight yet for the last two messages. - - // the first message was committed even though we waited to pull the next message only because of touches - KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 3, sendCompleteCount); - readNextNStreams(trafficSource, keysReceived,2, 1 ); - // - - KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 4, sendCompleteCount); - - // wait long enough to call the BlockingTrafficSource so that the kafka consumer will fall out - // and make sure that we have only committed the one message within the delayed scheduler above - // (we should have definitely hit that since readNextNStreams() will block - Thread.sleep(2* MAX_POLL_INTERVAL_MS); - readNextNStreams(trafficSource, keysReceived, 3, 1); - Assertions.assertEquals(4, keysReceived.size()); - - - - - - // the first message was committed even though we waited to pull the next message only because of touches - KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 3, sendCompleteCount); - readNextNStreams(trafficSource, keysReceived,2, 1 ); - // - - KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 4, sendCompleteCount); + trafficSource.stopReadsPast(Instant.MAX); + for (int i=0; i<2; ++i) { + KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1 + i, sendCompleteCount).get(); + } + readNextNStreams(trafficSource, keysReceived, 1, 1); + + trafficSource.commitTrafficStream(keysReceived.get(0)); + log.info("Called commitTrafficStream but waiting long enough for the client to leave the group. " + + "That will make the previous commit a 'zombie-commit' that should easily be dropped."); + + log.info("1 message was committed, but not synced, 1 message is being processed." + + "wait long enough to fall out of the group before we can commit"); + Thread.sleep(2*MAX_POLL_INTERVAL_MS); + + var keysReceivedUntilDrop1 = keysReceived; + keysReceived = new ArrayList<>(); + + log.info("re-establish a client connection so that the following commit will work"); + log.atInfo().setMessage(()->"1 ..."+renderNextCommitsAsString()).log(); + readNextNStreams(trafficSource, keysReceived, 0, 1); + log.atInfo().setMessage(()->"2 ..."+renderNextCommitsAsString()).log(); + + log.info("wait long enough to fall out of the group again"); + Thread.sleep(2*MAX_POLL_INTERVAL_MS); + + var keysReceivedUntilDrop2 = keysReceived; + keysReceived = new ArrayList<>(); + log.atInfo().setMessage(()->"re-establish... 3 ..."+renderNextCommitsAsString()).log(); + readNextNStreams(trafficSource, keysReceived, 0, 1); + trafficSource.commitTrafficStream(keysReceivedUntilDrop1.get(1)); + log.atInfo().setMessage(()->"re-establish... 4 ..."+renderNextCommitsAsString()).log(); + readNextNStreams(trafficSource, keysReceived, 1, 1); + log.atInfo().setMessage(()->"5 ..."+renderNextCommitsAsString()).log(); + + Thread.sleep(2*MAX_POLL_INTERVAL_MS); + readNextNStreams(trafficSource, keysReceived, 0, 3); + log.atInfo().setMessage(()->"6 ..."+renderNextCommitsAsString()).log(); + } - // wait long enough to call the BlockingTrafficSource so that the kafka consumer will fall out - // and make sure that we have only committed the one message within the delayed scheduler above - // (we should have definitely hit that since readNextNStreams() will block - Thread.sleep(2* MAX_POLL_INTERVAL_MS); - readNextNStreams(trafficSource, keysReceived, 3, 1); - Assertions.assertEquals(4, keysReceived.size()); + private String renderNextCommitsAsString() { + return "nextCommits="+kafkaSource.nextSetOfCommitsMap.entrySet().stream() + .map(kvp->kvp.getKey()+"->"+kvp.getValue()).collect(Collectors.joining(",")); + } + private void stopReadsPast(int i) { + trafficSource.stopReadsPast(Instant.EPOCH.plus(Duration.ofDays(i))); } - + @SneakyThrows private static void readNextNStreams(BlockingTrafficSource kafkaSource, List keysReceived, int from, int count) { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java index 329bf90ac..691439ff1 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTestUtils.java @@ -18,6 +18,7 @@ import org.testcontainers.utility.DockerImageName; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.time.Instant; import java.util.Properties; import java.util.concurrent.Future; @@ -56,7 +57,7 @@ static String getConnectionId(int i) { static TrafficStream makeTestTrafficStream(Instant t, int i) { var timestamp = Timestamp.newBuilder() - .setSeconds(t.getEpochSecond()) + .setSeconds(t.plus(Duration.ofDays(i)).getEpochSecond()) .setNanos(t.getNano()) .build(); var tsb = TrafficStream.newBuilder() diff --git a/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties b/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties index 52d5190a1..9098da413 100644 --- a/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties +++ b/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties @@ -14,3 +14,7 @@ appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t # of the logs for tests logger.OutputTupleJsonLogger.name = OutputTupleJsonLogger logger.OutputTupleJsonLogger.level = OFF + +logger.KPC.name = org.opensearch.migrations.replay.kafka.KafkaProtobufConsumer +logger.KPC.level = DEBUG +logger.KPC.appenderRef.stdout.ref = Console From b9247f682aa66a26d638e5bf46ddd7499c11496a Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Mon, 20 Nov 2023 14:53:56 -0500 Subject: [PATCH 5/9] Refactor KafkaProtobufConsumer into multiple pieces. Normally, I'd try to keep renames in place, but the file has been separated into multiple pieces and once commit tracking is fully in place, the result will be very different from the starting point anyway. That said, at present, all of the files under the replay/kafka directory (other than BehavioralPolicy) are offshoots of the old KafkaProtobufConsumer. I'm doing the refactoring to separate the concerns of kafka from the concerns of a traffic source. Signed-off-by: Greg Schohn --- .../replay/TrafficCaptureSourceFactory.java | 4 +- .../replay/kafka/KafkaCommitOffsetData.java | 18 + .../replay/kafka/KafkaProtobufConsumer.java | 386 ------------------ .../kafka/KafkaTrafficCaptureSource.java | 220 ++++++++++ .../replay/kafka/OffsetLifecycleTracker.java | 42 ++ .../replay/kafka/TrackingKafkaConsumer.java | 184 +++++++++ .../TrafficStreamKeyWithKafkaRecordId.java | 27 ++ .../traffic/source/BlockingTrafficSource.java | 6 +- .../KafkaRestartingTrafficReplayerTest.java | 10 +- .../replay/kafka/KafkaKeepAliveTests.java | 18 +- ...afkaTrafficCaptureSourceLongTermTest.java} | 32 +- ...ava => KafkaTrafficCaptureSourceTest.java} | 14 +- 12 files changed, 519 insertions(+), 442 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java delete mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java rename TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/{KafkaProtobufConsumerLongTermTest.java => KafkaTrafficCaptureSourceLongTermTest.java} (72%) rename TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/{KafkaProtobufConsumerTest.java => KafkaTrafficCaptureSourceTest.java} (94%) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java index 0d90f062e..93d1bb664 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/TrafficCaptureSourceFactory.java @@ -2,7 +2,7 @@ import lombok.extern.slf4j.Slf4j; import org.opensearch.migrations.replay.kafka.KafkaBehavioralPolicy; -import org.opensearch.migrations.replay.kafka.KafkaProtobufConsumer; +import org.opensearch.migrations.replay.kafka.KafkaTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.InputStreamOfTraffic; @@ -32,7 +32,7 @@ private TrafficCaptureSourceFactory() {} } if (isKafkaActive) { - return KafkaProtobufConsumer.buildKafkaConsumer(appParams.kafkaTrafficBrokers, appParams.kafkaTrafficTopic, + return KafkaTrafficCaptureSource.buildKafkaConsumer(appParams.kafkaTrafficBrokers, appParams.kafkaTrafficTopic, appParams.kafkaTrafficGroupId, appParams.kafkaTrafficEnableMSKAuth, appParams.kafkaTrafficPropertyFile, Clock.systemUTC(), new KafkaBehavioralPolicy()); } else if (isInputFileActive) { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java new file mode 100644 index 000000000..fb8eb4a96 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java @@ -0,0 +1,18 @@ +package org.opensearch.migrations.replay.kafka; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +public interface KafkaCommitOffsetData { + int getPartition(); + long getOffset(); + int getGeneration(); + + @Getter + @AllArgsConstructor + public static class PojoKafkaCommitOffsetData implements KafkaCommitOffsetData { + final int generation; + final int partition; + final long offset; + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java deleted file mode 100644 index 204fe9e26..000000000 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumer.java +++ /dev/null @@ -1,386 +0,0 @@ -package org.opensearch.migrations.replay.kafka; - -import com.google.protobuf.InvalidProtocolBufferException; -import lombok.EqualsAndHashCode; -import lombok.NonNull; -import lombok.ToString; -import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.opensearch.migrations.coreutils.MetricsAttributeKey; -import org.opensearch.migrations.coreutils.MetricsEvent; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.common.TopicPartition; -import org.opensearch.migrations.coreutils.MetricsLogger; -import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; -import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamWithKey; -import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; -import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; -import org.opensearch.migrations.trafficcapture.protos.TrafficStream; -import org.slf4j.event.Level; - -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.time.Clock; -import java.time.Duration; -import java.time.Instant; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.PriorityQueue; -import java.util.Properties; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; - -/** - * Adapt a Kafka stream into a TrafficCaptureSource. - * - * Notice that there's a critical gap between how Kafka accepts commits and how the - * BlockingTrafficSource throttles calls to Kafka. The BlockingTrafficSource may - * block calls to readNextTrafficStreamChunk() until some time window elapses. This - * could be a very large window in cases where there were long gaps between recorded - * requests from the capturing proxy. For example, if a TrafficStream is read and it - * that stream is scheduled to be run one hour later, readNextTrafficStreamChunk() - * may not be called for almost an hour. By design, we're not calling Kafka to pull - * any more messages since we know that we don't have work to do for an hour. Shortly - * after the hour of waiting begins, Kakfa will notice that this application is no - * longer calling poll and will kick the consumer out of the client group. - * - * See - * ... - * - * "Basically if you don't call poll at least as frequently as the configured max interval, - * then the client will proactively leave the group so that another consumer can take - * over its partitions. When this happens, you may see an offset commit failure (as - * indicated by a CommitFailedException thrown from a call to commitSync())." - * - * Since the Kafka client requires all calls to be made from the same thread, we can't - * simply run a background job to keep the client warm. We need the caller to touch - * this object periodically to keep the connection alive. - */ -@Slf4j -public class KafkaProtobufConsumer implements ISimpleTrafficCaptureSource { - - public static final String MAX_POLL_INTERVAL_KEY = "max.poll.interval.ms"; - - @ToString(callSuper = true) - @EqualsAndHashCode(callSuper = true) - private static class TrafficStreamKeyWithKafkaRecordId extends PojoTrafficStreamKey { - private final int partition; - private final long offset; - private final int generation; - - TrafficStreamKeyWithKafkaRecordId(int generation, TrafficStream trafficStream, int partition, long offset) { - super(trafficStream); - this.generation = generation; - this.partition = partition; - this.offset = offset; - } - } - - private static class OffsetLifecycleTracker { - private final PriorityQueue pQueue = new PriorityQueue<>(); - private long cursorHighWatermark; - - private OffsetLifecycleTracker() { - } - - boolean isEmpty() { - return pQueue.isEmpty(); - } - - void add(long offset) { - cursorHighWatermark = offset; - pQueue.add(offset); - } - - Optional removeAndReturnNewHead(TrafficStreamKeyWithKafkaRecordId kafkaRecord) { - var offsetToRemove = kafkaRecord.offset; - var topCursor = pQueue.peek(); - var didRemove = pQueue.remove(offsetToRemove); - assert didRemove : "Expected all live records to have an entry and for them to be removed only once"; - if (topCursor == offsetToRemove) { - topCursor = Optional.ofNullable(pQueue.peek()) - .orElse(cursorHighWatermark+1); // most recent cursor was previously popped - log.atDebug().setMessage("Commit called for {}, and new topCursor={}") - .addArgument(offsetToRemove).addArgument(topCursor).log(); - return Optional.of(topCursor); - } else { - log.atDebug().setMessage("Commit called for {}, but topCursor={}") - .addArgument(offsetToRemove).addArgument(topCursor).log(); - return Optional.empty(); - } - } - } - - private static final MetricsLogger metricsLogger = new MetricsLogger("KafkaProtobufConsumer"); - - public static final Duration CONSUMER_POLL_TIMEOUT = Duration.ofSeconds(1); - - private final Clock clock; - private final Consumer kafkaConsumer; - final ConcurrentHashMap partitionToOffsetLifecycleTrackerMap; - // loosening visibility so that a unit test can read this - final ConcurrentHashMap nextSetOfCommitsMap; - private final Object offsetLifecycleLock = new Object(); - private final String topic; - private final Duration keepAliveInterval; - private final KafkaBehavioralPolicy behavioralPolicy; - private final AtomicInteger trafficStreamsRead; - private Instant lastPollTimestamp; - private int consumerConnectionGeneration; - - public KafkaProtobufConsumer(Consumer kafkaConsumer, String topic, Duration keepAliveInterval) { - this(kafkaConsumer, topic, keepAliveInterval, Clock.systemUTC(), new KafkaBehavioralPolicy()); - } - - public KafkaProtobufConsumer(Consumer kafkaConsumer, - @NonNull String topic, - Duration keepAliveInterval, - Clock clock, - @NonNull KafkaBehavioralPolicy behavioralPolicy) - { - this.kafkaConsumer = kafkaConsumer; - this.topic = topic; - this.keepAliveInterval = keepAliveInterval; - log.error("keepAliveInterval="+keepAliveInterval); - this.behavioralPolicy = behavioralPolicy; - kafkaConsumer.subscribe(Collections.singleton(topic)); - trafficStreamsRead = new AtomicInteger(); - this.clock = clock; - - partitionToOffsetLifecycleTrackerMap = new ConcurrentHashMap<>(); - nextSetOfCommitsMap = new ConcurrentHashMap<>(); - } - - public static KafkaProtobufConsumer buildKafkaConsumer(@NonNull String brokers, - @NonNull String topic, - @NonNull String groupId, - boolean enableMSKAuth, - String propertyFilePath, - Clock clock, - @NonNull KafkaBehavioralPolicy behavioralPolicy) - throws IOException - { - var kafkaProps = buildKafkaProperties(brokers, groupId, enableMSKAuth, propertyFilePath); - var pollPeriod = Duration.ofMillis(Integer.valueOf((String)kafkaProps.get(MAX_POLL_INTERVAL_KEY))); - var keepAlivePeriod = getKeepAlivePeriodFromPollPeriod(pollPeriod); - return new KafkaProtobufConsumer(new KafkaConsumer<>(kafkaProps), topic, keepAlivePeriod, - clock, behavioralPolicy); - } - - private static Duration getKeepAlivePeriodFromPollPeriod(Duration pollPeriod) { - return pollPeriod.dividedBy(2); - } - - public static Properties buildKafkaProperties(@NonNull String brokers, - @NonNull String groupId, - boolean enableMSKAuth, - String propertyFilePath) throws IOException { - var kafkaProps = new Properties(); - kafkaProps.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - kafkaProps.setProperty("value.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - kafkaProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - kafkaProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - if (propertyFilePath != null) { - try (InputStream input = new FileInputStream(propertyFilePath)) { - kafkaProps.load(input); - } catch (IOException ex) { - log.error("Unable to load properties from kafka properties file with path: {}", propertyFilePath); - throw ex; - } - } - // Required for using SASL auth with MSK public endpoint - if (enableMSKAuth) { - kafkaProps.setProperty("security.protocol", "SASL_SSL"); - kafkaProps.setProperty("sasl.mechanism", "AWS_MSK_IAM"); - kafkaProps.setProperty("sasl.jaas.config", "software.amazon.msk.auth.iam.IAMLoginModule required;"); - kafkaProps.setProperty("sasl.client.callback.handler.class", "software.amazon.msk.auth.iam.IAMClientCallbackHandler"); - } - kafkaProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers); - kafkaProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); - return kafkaProps; - } - - @Override - public void touch() { - log.error("TOUCH CALLED"); - try { - var records = kafkaConsumer.poll(Duration.ZERO); - log.atError().setMessage(()->"Polled "+records.count()+" records to keep the consumer alive").log(); - records.forEach(r -> { - try { - var tp = new TopicPartition(r.topic(), r.partition()); - log.atError().setMessage(()->"Resetting "+tp+" to offset="+r.offset()).log(); - kafkaConsumer.seek(tp, r.offset()); - } catch (IllegalStateException e) { - log.atWarn().setCause(e).setMessage(() -> "Caught exception while seeking. " + - "Ignoring so that other records can have their seeks readjusted.").log(); - } - }); - } catch (RuntimeException e) { - log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + - "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); - resetCommitInfo(); - } - safeCommitWithRetry(); - lastPollTimestamp = clock.instant(); - } - - /** - * If messages are outstanding, we need to keep the connection alive, otherwise, there's no - * reason to. It's OK to fall out of the group and rejoin once ready. - * @return - */ - @Override - public Optional getNextRequiredTouch() { - synchronized (offsetLifecycleLock) { - return partitionToOffsetLifecycleTrackerMap.isEmpty() ? Optional.empty() : - Optional.of(lastPollTimestamp.plus(keepAliveInterval)); - } - } - - @Override - @SuppressWarnings("unchecked") - public CompletableFuture> readNextTrafficStreamChunk() { - return CompletableFuture.supplyAsync(this::readNextTrafficStreamSynchronously); - } - - public List readNextTrafficStreamSynchronously() { - try { - ConsumerRecords records; - safeCommitWithRetry(); - records = safePollWithSwallowedRuntimeExceptions(); - Stream trafficStream = StreamSupport.stream(records.spliterator(), false) - .map(kafkaRecord -> { - try { - TrafficStream ts = TrafficStream.parseFrom(kafkaRecord.value()); - // Ensure we increment trafficStreamsRead even at a higher log level - metricsLogger.atSuccess(MetricsEvent.PARSED_TRAFFIC_STREAM_FROM_KAFKA) - .setAttribute(MetricsAttributeKey.CONNECTION_ID, ts.getConnectionId()) - .setAttribute(MetricsAttributeKey.TOPIC_NAME, this.topic) - .setAttribute(MetricsAttributeKey.SIZE_IN_BYTES, ts.getSerializedSize()).emit(); - addOffset(kafkaRecord.partition(), kafkaRecord.offset()); - var key = new TrafficStreamKeyWithKafkaRecordId(consumerConnectionGeneration, ts, - kafkaRecord.partition(), kafkaRecord.offset()); - log.atTrace().setMessage(()->"Parsed traffic stream #{}: {} {}") - .addArgument(trafficStreamsRead.incrementAndGet()) - .addArgument(key) - .addArgument(ts) - .log(); - return (ITrafficStreamWithKey) new PojoTrafficStreamWithKey(ts, key); - } catch (InvalidProtocolBufferException e) { - RuntimeException recordError = behavioralPolicy.onInvalidKafkaRecord(kafkaRecord, e); - metricsLogger.atError(MetricsEvent.PARSING_TRAFFIC_STREAM_FROM_KAFKA_FAILED, recordError) - .setAttribute(MetricsAttributeKey.TOPIC_NAME, this.topic).emit(); - if (recordError != null) { - throw recordError; - } - return null; - } - }).filter(Objects::nonNull); - return trafficStream.collect(Collectors.toList()); - } catch (Exception e) { - log.error("Terminating Kafka traffic stream"); - throw e; - } - } - - private void safeCommitWithRetry() { - synchronized (offsetLifecycleLock) { - try { - if (!nextSetOfCommitsMap.isEmpty()) { - log.atDebug().setMessage(() -> "Committing " + nextSetOfCommitsMap).log(); - kafkaConsumer.commitSync(nextSetOfCommitsMap); - log.atDebug().setMessage(() -> "Done committing " + nextSetOfCommitsMap).log(); - } - nextSetOfCommitsMap.clear(); - } catch (RuntimeException e) { - log.atWarn().setCause(e) - .setMessage(() -> "Error while committing. Purging all commit points since another consumer " + - "may have already begun processing messages BEFORE those commits. Commits being " + - "discarded: " + nextSetOfCommitsMap.entrySet().stream() - .map(kvp -> kvp.getKey() + "->" + kvp.getValue()).collect(Collectors.joining(","))) - .log(); - resetCommitInfo(); - } - } - } - - private void resetCommitInfo() { - ++consumerConnectionGeneration; - nextSetOfCommitsMap.clear(); - partitionToOffsetLifecycleTrackerMap.clear(); - - } - - private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { - try { - lastPollTimestamp = clock.instant(); - var records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT); - log.atLevel(records.isEmpty()? Level.TRACE:Level.INFO) - .setMessage(()->"Kafka consumer poll has fetched "+records.count()+" records").log(); - log.atDebug().setMessage(()->"All positions: {"+kafkaConsumer.assignment().stream() - .map(tp->tp+": "+kafkaConsumer.position(tp)).collect(Collectors.joining(",")) + "}").log(); - log.atDebug().setMessage(()->"All COMMITTED positions: {"+kafkaConsumer.assignment().stream() - .map(tp->tp+": "+kafkaConsumer.committed(tp)).collect(Collectors.joining(",")) + "}").log(); - return records; - } catch (RuntimeException e) { - log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + - "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); - resetCommitInfo(); - return new ConsumerRecords<>(Collections.emptyMap()); - } - } - - private void addOffset(int partition, long offset) { - synchronized (offsetLifecycleLock) { - var offsetTracker = partitionToOffsetLifecycleTrackerMap.computeIfAbsent(partition, p -> - new OffsetLifecycleTracker()); - offsetTracker.add(offset); - } - } - - @Override - public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { - if (!(trafficStreamKey instanceof TrafficStreamKeyWithKafkaRecordId)) { - throw new IllegalArgumentException("Expected key of type "+TrafficStreamKeyWithKafkaRecordId.class+ - " but received "+trafficStreamKey+" (of type="+trafficStreamKey.getClass()+")"); - } - var kafkaTsk = (TrafficStreamKeyWithKafkaRecordId) trafficStreamKey; - if (kafkaTsk.generation != consumerConnectionGeneration) { - log.atWarn().setMessage(()->"trafficKey's generation (" + kafkaTsk.generation + ") is not current (" + - consumerConnectionGeneration + "). Dropping this commit request since the record would have " + - "been or will be handled again by a current consumer within this process or another. Full key=" + - kafkaTsk).log(); - return; - } - var p = kafkaTsk.partition; - Optional newHeadValue; - synchronized (offsetLifecycleLock) { - var tracker = partitionToOffsetLifecycleTrackerMap.get(p); - newHeadValue = tracker.removeAndReturnNewHead(kafkaTsk); - newHeadValue.ifPresent(o -> { - if (tracker.isEmpty()) { - partitionToOffsetLifecycleTrackerMap.remove(p); - } - nextSetOfCommitsMap.put(new TopicPartition(topic, p), new OffsetAndMetadata(o)); - }); - } - } - - @Override - public void close() throws IOException { - kafkaConsumer.close(); - log.info("Kafka consumer closed successfully."); - } -} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java new file mode 100644 index 000000000..1f99a177e --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java @@ -0,0 +1,220 @@ +package org.opensearch.migrations.replay.kafka; + +import com.google.protobuf.InvalidProtocolBufferException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.opensearch.migrations.coreutils.MetricsAttributeKey; +import org.opensearch.migrations.coreutils.MetricsEvent; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.opensearch.migrations.coreutils.MetricsLogger; +import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamWithKey; +import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; +import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; +import org.opensearch.migrations.trafficcapture.protos.TrafficStream; +import org.slf4j.event.Level; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +/** + * Adapt a Kafka stream into a TrafficCaptureSource. + * + * Notice that there's a critical gap between how Kafka accepts commits and how the + * BlockingTrafficSource throttles calls to Kafka. The BlockingTrafficSource may + * block calls to readNextTrafficStreamChunk() until some time window elapses. This + * could be a very large window in cases where there were long gaps between recorded + * requests from the capturing proxy. For example, if a TrafficStream is read and it + * that stream is scheduled to be run one hour later, readNextTrafficStreamChunk() + * may not be called for almost an hour. By design, we're not calling Kafka to pull + * any more messages since we know that we don't have work to do for an hour. Shortly + * after the hour of waiting begins, Kakfa will notice that this application is no + * longer calling poll and will kick the consumer out of the client group. + * + * See + * ... + * + * "Basically if you don't call poll at least as frequently as the configured max interval, + * then the client will proactively leave the group so that another consumer can take + * over its partitions. When this happens, you may see an offset commit failure (as + * indicated by a CommitFailedException thrown from a call to commitSync())." + * + * Since the Kafka client requires all calls to be made from the same thread, we can't + * simply run a background job to keep the client warm. We need the caller to touch + * this object periodically to keep the connection alive. + */ +@Slf4j +public class KafkaTrafficCaptureSource implements ISimpleTrafficCaptureSource { + + public static final String MAX_POLL_INTERVAL_KEY = "max.poll.interval.ms"; + private static final MetricsLogger metricsLogger = new MetricsLogger("KafkaProtobufConsumer"); + + + private final Clock clock; + private final KafkaBehavioralPolicy behavioralPolicy; + private final AtomicLong trafficStreamsRead; + final TrackingKafkaConsumer workingState; + + public KafkaTrafficCaptureSource(Consumer kafkaConsumer, String topic, Duration keepAliveInterval) { + this(kafkaConsumer, topic, keepAliveInterval, Clock.systemUTC(), new KafkaBehavioralPolicy()); + } + + public KafkaTrafficCaptureSource(Consumer kafkaConsumer, + @NonNull String topic, + Duration keepAliveInterval, + Clock clock, + @NonNull KafkaBehavioralPolicy behavioralPolicy) + { + workingState = new TrackingKafkaConsumer(kafkaConsumer, topic, keepAliveInterval, clock); + this.behavioralPolicy = behavioralPolicy; + kafkaConsumer.subscribe(Collections.singleton(topic), workingState); + trafficStreamsRead = new AtomicLong(); + this.clock = clock; + } + + public static KafkaTrafficCaptureSource buildKafkaConsumer(@NonNull String brokers, + @NonNull String topic, + @NonNull String groupId, + boolean enableMSKAuth, + String propertyFilePath, + Clock clock, + @NonNull KafkaBehavioralPolicy behavioralPolicy) + throws IOException + { + var kafkaProps = buildKafkaProperties(brokers, groupId, enableMSKAuth, propertyFilePath); + var pollPeriod = Duration.ofMillis(Integer.valueOf((String)kafkaProps.get(MAX_POLL_INTERVAL_KEY))); + var keepAlivePeriod = getKeepAlivePeriodFromPollPeriod(pollPeriod); + return new KafkaTrafficCaptureSource(new KafkaConsumer<>(kafkaProps), topic, keepAlivePeriod, + clock, behavioralPolicy); + } + + private static Duration getKeepAlivePeriodFromPollPeriod(Duration pollPeriod) { + return pollPeriod.dividedBy(2); + } + + public static Properties buildKafkaProperties(@NonNull String brokers, + @NonNull String groupId, + boolean enableMSKAuth, + String propertyFilePath) throws IOException { + var kafkaProps = new Properties(); + kafkaProps.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + kafkaProps.setProperty("value.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + kafkaProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + kafkaProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + if (propertyFilePath != null) { + try (InputStream input = new FileInputStream(propertyFilePath)) { + kafkaProps.load(input); + } catch (IOException ex) { + log.error("Unable to load properties from kafka properties file with path: {}", propertyFilePath); + throw ex; + } + } + // Required for using SASL auth with MSK public endpoint + if (enableMSKAuth) { + kafkaProps.setProperty("security.protocol", "SASL_SSL"); + kafkaProps.setProperty("sasl.mechanism", "AWS_MSK_IAM"); + kafkaProps.setProperty("sasl.jaas.config", "software.amazon.msk.auth.iam.IAMLoginModule required;"); + kafkaProps.setProperty("sasl.client.callback.handler.class", "software.amazon.msk.auth.iam.IAMClientCallbackHandler"); + } + kafkaProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers); + kafkaProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); + return kafkaProps; + } + + @Override + public void touch() { + workingState.touch(); + } + + /** + * If messages are outstanding, we need to keep the connection alive, otherwise, there's no + * reason to. It's OK to fall out of the group and rejoin once ready. + * @return + */ + @Override + public Optional getNextRequiredTouch() { + return workingState.getNextRequiredTouch(); + } + + @Override + @SuppressWarnings("unchecked") + public CompletableFuture> readNextTrafficStreamChunk() { + log.atTrace().setMessage("readNextTrafficStreamChunk()").log(); + return CompletableFuture.supplyAsync(() -> { + log.atTrace().setMessage("async...readNextTrafficStreamChunk()").log(); + return readNextTrafficStreamSynchronously(); + }); + } + + public List readNextTrafficStreamSynchronously() { + log.atTrace().setMessage("readNextTrafficStreamSynchronously()").log(); + try { + var records = workingState.getNextBatchOfRecords(); + Stream trafficStream = StreamSupport.stream(records.spliterator(), false) + .map(kafkaRecord -> { + try { + TrafficStream ts = TrafficStream.parseFrom(kafkaRecord.value()); + // Ensure we increment trafficStreamsRead even at a higher log level + metricsLogger.atSuccess(MetricsEvent.PARSED_TRAFFIC_STREAM_FROM_KAFKA) + .setAttribute(MetricsAttributeKey.CONNECTION_ID, ts.getConnectionId()) + .setAttribute(MetricsAttributeKey.TOPIC_NAME, workingState.topic) + .setAttribute(MetricsAttributeKey.SIZE_IN_BYTES, ts.getSerializedSize()).emit(); + var key = workingState.createAndTrackKey(kafkaRecord.partition(), kafkaRecord.offset(), + ck -> new TrafficStreamKeyWithKafkaRecordId(ts, ck)); + log.atTrace().setMessage(()->"Parsed traffic stream #{}: {} {}") + .addArgument(trafficStreamsRead.incrementAndGet()) + .addArgument(key) + .addArgument(ts) + .log(); + return (ITrafficStreamWithKey) new PojoTrafficStreamWithKey(ts, key); + } catch (InvalidProtocolBufferException e) { + RuntimeException recordError = behavioralPolicy.onInvalidKafkaRecord(kafkaRecord, e); + metricsLogger.atError(MetricsEvent.PARSING_TRAFFIC_STREAM_FROM_KAFKA_FAILED, recordError) + .setAttribute(MetricsAttributeKey.TOPIC_NAME, workingState.topic).emit(); + if (recordError != null) { + throw recordError; + } + return null; + } + }).filter(Objects::nonNull); + return trafficStream.collect(Collectors.toList()); + } catch (Exception e) { + log.error("Terminating Kafka traffic stream"); + throw e; + } + } + + @Override + public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { + if (!(trafficStreamKey instanceof TrafficStreamKeyWithKafkaRecordId)) { + throw new IllegalArgumentException("Expected key of type "+TrafficStreamKeyWithKafkaRecordId.class+ + " but received "+trafficStreamKey+" (of type="+trafficStreamKey.getClass()+")"); + } + var kafkaTsk = (TrafficStreamKeyWithKafkaRecordId) trafficStreamKey; + + } + + @Override + public void close() throws IOException { + workingState.close(); + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java new file mode 100644 index 000000000..1ec27944f --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java @@ -0,0 +1,42 @@ +package org.opensearch.migrations.replay.kafka; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Optional; +import java.util.PriorityQueue; + +@Slf4j +class OffsetLifecycleTracker { + private final PriorityQueue pQueue = new PriorityQueue<>(); + private long cursorHighWatermark; + + OffsetLifecycleTracker() { + } + + boolean isEmpty() { + return pQueue.isEmpty(); + } + + void add(long offset) { + cursorHighWatermark = offset; + pQueue.add(offset); + } + + Optional removeAndReturnNewHead(KafkaCommitOffsetData kafkaRecord) { + var offsetToRemove = kafkaRecord.getOffset(); + var topCursor = pQueue.peek(); + var didRemove = pQueue.remove(offsetToRemove); + assert didRemove : "Expected all live records to have an entry and for them to be removed only once"; + if (topCursor == offsetToRemove) { + topCursor = Optional.ofNullable(pQueue.peek()) + .orElse(cursorHighWatermark + 1); // most recent cursor was previously popped + log.atDebug().setMessage("Commit called for {}, and new topCursor={}") + .addArgument(offsetToRemove).addArgument(topCursor).log(); + return Optional.of(topCursor); + } else { + log.atDebug().setMessage("Commit called for {}, but topCursor={}") + .addArgument(offsetToRemove).addArgument(topCursor).log(); + return Optional.empty(); + } + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java new file mode 100644 index 000000000..0e7ddb6ef --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java @@ -0,0 +1,184 @@ +package org.opensearch.migrations.replay.kafka; + +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.event.Level; +//import org.opensearch.migrations.trafficcapture.protos.TrafficStream; + +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; +import java.util.stream.Collectors; + +@Slf4j +public class TrackingKafkaConsumer implements ConsumerRebalanceListener { + + public static final Duration CONSUMER_POLL_TIMEOUT = Duration.ofSeconds(1); + + private final Consumer kafkaConsumer; + + final String topic; + private final Clock clock; + final ConcurrentHashMap partitionToOffsetLifecycleTrackerMap; + // loosening visibility so that a unit test can read this + final ConcurrentHashMap nextSetOfCommitsMap; + private final Duration keepAliveInterval; + private final Object offsetLifecycleLock = new Object(); + private int consumerConnectionGeneration; + private Instant lastPollTimestamp; + + public TrackingKafkaConsumer(Consumer kafkaConsumer, String topic, + Duration keepAliveInterval, Clock c) { + this.kafkaConsumer = kafkaConsumer; + this.topic = topic; + this.clock = c; + this.partitionToOffsetLifecycleTrackerMap = new ConcurrentHashMap<>(); + this.nextSetOfCommitsMap = new ConcurrentHashMap<>(); + this.keepAliveInterval = keepAliveInterval; + log.error("keepAliveInterval="+keepAliveInterval); + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + log.atWarn().setCause(new Exception()).setMessage(()->this+"partitions revoked for "+partitions.stream() + .map(p->p+"").collect(Collectors.joining(","))).log(); + safeCommitWithRetry(); + } + + @Override public void onPartitionsAssigned(Collection partitions) { + log.atWarn().setCause(new Exception()).setMessage(()->this+"partitions added for "+partitions.stream() + .map(p->p+"").collect(Collectors.joining(","))).log(); + } + + public void close() { + kafkaConsumer.close(); + log.info("Kafka consumer closed successfully."); + } + + public Optional getNextRequiredTouch() { + synchronized (offsetLifecycleLock) { + return partitionToOffsetLifecycleTrackerMap.isEmpty() ? Optional.empty() : + Optional.of(lastPollTimestamp.plus(keepAliveInterval)); + } + } + + public void touch() { + log.error("TOUCH CALLED"); + try { + //kafkaConsumer.pause(); + var records = kafkaConsumer.poll(Duration.ZERO); + log.atError().setMessage(()->"Polled "+records.count()+" records to keep the consumer alive").log(); + records.forEach(r -> { + try { + var tp = new TopicPartition(r.topic(), r.partition()); + log.atError().setMessage(()->"Resetting "+tp+" to offset="+r.offset()).log(); + kafkaConsumer.seek(tp, r.offset()); + } catch (IllegalStateException e) { + log.atWarn().setCause(e).setMessage(() -> "Caught exception while seeking. " + + "Ignoring so that other records can have their seeks readjusted.").log(); + } + }); + } catch (RuntimeException e) { + log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + + "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); + resetCommitInfo(); + } + safeCommitWithRetry(); + lastPollTimestamp = clock.instant(); + } + + public K createAndTrackKey(int partition, long offset, + Function keyFactory) { + synchronized (offsetLifecycleLock) { + var offsetTracker = partitionToOffsetLifecycleTrackerMap.computeIfAbsent(partition, + p -> new OffsetLifecycleTracker()); + offsetTracker.add(offset); + } + return keyFactory.apply(new KafkaCommitOffsetData.PojoKafkaCommitOffsetData(consumerConnectionGeneration, + partition, offset)); + } + + ConsumerRecords getNextBatchOfRecords() { + safeCommitWithRetry(); + return safePollWithSwallowedRuntimeExceptions(); + } + + private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { + try { + lastPollTimestamp = clock.instant(); + var records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT); + log.atLevel(records.isEmpty()? Level.TRACE:Level.INFO) + .setMessage(()->"Kafka consumer poll has fetched "+records.count()+" records").log(); + log.atDebug().setMessage(()->"All positions: {"+kafkaConsumer.assignment().stream() + .map(tp->tp+": "+kafkaConsumer.position(tp)).collect(Collectors.joining(",")) + "}").log(); + log.atDebug().setMessage(()->"All COMMITTED positions: {"+kafkaConsumer.assignment().stream() + .map(tp->tp+": "+kafkaConsumer.committed(tp)).collect(Collectors.joining(",")) + "}").log(); + return records; + } catch (RuntimeException e) { + log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + + "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); + resetCommitInfo(); + return new ConsumerRecords<>(Collections.emptyMap()); + } + } + + void commitKafkaKey(KafkaCommitOffsetData kafkaTsk) { + if (kafkaTsk.getGeneration() != consumerConnectionGeneration) { + log.atWarn().setMessage(()->"trafficKey's generation (" + kafkaTsk.getGeneration() + ") is not current (" + + consumerConnectionGeneration + "). Dropping this commit request since the record would have " + + "been or will be handled again by a current consumer within this process or another. Full key=" + + kafkaTsk).log(); + return; + } + var p = kafkaTsk.getPartition(); + Optional newHeadValue; + synchronized (offsetLifecycleLock) { + var tracker = partitionToOffsetLifecycleTrackerMap.get(p); + newHeadValue = tracker.removeAndReturnNewHead(kafkaTsk); + newHeadValue.ifPresent(o -> { + if (tracker.isEmpty()) { + partitionToOffsetLifecycleTrackerMap.remove(p); + } + nextSetOfCommitsMap.put(new TopicPartition(topic, p), new OffsetAndMetadata(o)); + }); + } + } + + private void safeCommitWithRetry() { + synchronized (offsetLifecycleLock) { + try { + if (!nextSetOfCommitsMap.isEmpty()) { + log.atDebug().setMessage(() -> "Committing " + nextSetOfCommitsMap).log(); + kafkaConsumer.commitSync(nextSetOfCommitsMap); + log.atDebug().setMessage(() -> "Done committing " + nextSetOfCommitsMap).log(); + } + nextSetOfCommitsMap.clear(); + } catch (RuntimeException e) { + log.atWarn().setCause(e) + .setMessage(() -> "Error while committing. Purging all commit points since another consumer " + + "may have already begun processing messages BEFORE those commits. Commits being " + + "discarded: " + nextSetOfCommitsMap.entrySet().stream() + .map(kvp -> kvp.getKey() + "->" + kvp.getValue()).collect(Collectors.joining(","))) + .log(); + resetCommitInfo(); + } + } + } + + private void resetCommitInfo() { + ++consumerConnectionGeneration; + nextSetOfCommitsMap.clear(); + partitionToOffsetLifecycleTrackerMap.clear(); + + } + +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java new file mode 100644 index 000000000..aa86e96c7 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrafficStreamKeyWithKafkaRecordId.java @@ -0,0 +1,27 @@ +package org.opensearch.migrations.replay.kafka; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamKey; +import org.opensearch.migrations.trafficcapture.protos.TrafficStream; + +@ToString(callSuper = true) +@EqualsAndHashCode(callSuper = true) +@Getter +class TrafficStreamKeyWithKafkaRecordId extends PojoTrafficStreamKey implements KafkaCommitOffsetData { + private final int generation; + private final int partition; + private final long offset; + + TrafficStreamKeyWithKafkaRecordId(TrafficStream trafficStream, KafkaCommitOffsetData ok) { + this(trafficStream, ok.getGeneration(), ok.getPartition(), ok.getOffset()); + } + + TrafficStreamKeyWithKafkaRecordId(TrafficStream trafficStream, int generation, int partition, long offset) { + super(trafficStream); + this.generation = generation; + this.partition = partition; + this.offset = offset; + } +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java index ab36d383f..42e106cf0 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java @@ -90,9 +90,13 @@ public Duration getBufferTimeWindow() { @Override public CompletableFuture> readNextTrafficStreamChunk() { + log.info("BlockingTrafficSource::readNext"); var trafficStreamListFuture = CompletableFuture .supplyAsync(this::blockIfNeeded, task -> new Thread(task).start()) - .thenCompose(v->underlyingSource.readNextTrafficStreamChunk()); + .thenCompose(v->{ + log.info("BlockingTrafficSource::composing"); + return underlyingSource.readNextTrafficStreamChunk(); + }); return trafficStreamListFuture.whenComplete((v,t)->{ if (t != null) { return; diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java index bff5d2c9c..13562b0c5 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java @@ -11,10 +11,8 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; -import org.opensearch.migrations.replay.kafka.KafkaProtobufConsumer; +import org.opensearch.migrations.replay.kafka.KafkaTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.replay.traffic.source.TrafficStreamWithEmbeddedKey; @@ -99,14 +97,14 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(streamAndConsumer.numHttpTransactions, httpServer.localhostEndpoint(), new CounterLimitedReceiverFactory(), () -> new SentinelSensingTrafficSource( - new KafkaProtobufConsumer(buildKafkaConsumer(), TEST_TOPIC_NAME, + new KafkaTrafficCaptureSource(buildKafkaConsumer(), TEST_TOPIC_NAME, Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS)))); log.error("done"); } @SneakyThrows private KafkaConsumer buildKafkaConsumer() { - var kafkaConsumerProps = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), + var kafkaConsumerProps = KafkaTrafficCaptureSource.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), TEST_GROUP_CONSUMER_ID, false, null); kafkaConsumerProps.setProperty("max.poll.interval.ms", DEFAULT_POLL_INTERVAL_MS+""); var kafkaConsumer = new KafkaConsumer(kafkaConsumerProps); @@ -176,7 +174,7 @@ Producer buildKafkaProducer() { throw Lombok.sneakyThrow(e); } }); - return () -> new KafkaProtobufConsumer(kafkaConsumer, TEST_TOPIC_NAME, + return () -> new KafkaTrafficCaptureSource(kafkaConsumer, TEST_TOPIC_NAME, Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS)); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java index 5d21d56ab..8739dd21f 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java @@ -1,12 +1,10 @@ package org.opensearch.migrations.replay.kafka; -import com.beust.ah.A; import lombok.Lombok; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.Producer; -import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; @@ -18,14 +16,12 @@ import org.testcontainers.junit.jupiter.Testcontainers; import org.testcontainers.utility.DockerImageName; -import java.io.IOException; import java.time.Duration; import java.time.Instant; import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.Properties; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -51,7 +47,7 @@ public class KafkaKeepAliveTests { // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility private KafkaContainer embeddedKafkaBroker = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); - private KafkaProtobufConsumer kafkaSource; + private KafkaTrafficCaptureSource kafkaSource; /** * Setup the test case where we've produced and received 1 message, but have not yet committed it. @@ -66,15 +62,15 @@ private void setupTestCase() throws Exception { KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 0, sendCompleteCount).get(); Assertions.assertEquals(1, sendCompleteCount.get()); - this.kafkaProperties = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), + this.kafkaProperties = KafkaTrafficCaptureSource.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), TEST_GROUP_CONSUMER_ID, false, null); - Assertions.assertNull(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)); + Assertions.assertNull(kafkaProperties.get(KafkaTrafficCaptureSource.MAX_POLL_INTERVAL_KEY)); - kafkaProperties.put(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY, MAX_POLL_INTERVAL_MS+""); + kafkaProperties.put(KafkaTrafficCaptureSource.MAX_POLL_INTERVAL_KEY, MAX_POLL_INTERVAL_MS+""); kafkaProperties.put(HEARTBEAT_INTERVAL_MS_KEY, HEARTBEAT_INTERVAL_MS+""); kafkaProperties.put("max.poll.records", 1); var kafkaConsumer = new KafkaConsumer(kafkaProperties); - this.kafkaSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)); + this.kafkaSource = new KafkaTrafficCaptureSource(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)); this.trafficSource = new BlockingTrafficSource(kafkaSource, Duration.ZERO); this.keysReceived = new ArrayList(); @@ -86,7 +82,7 @@ private void setupTestCase() throws Exception { @Test @Tag("longTest") public void testTimeoutsDontOccurForSlowPolls() throws Exception { - var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY)) + var pollIntervalMs = Optional.ofNullable(kafkaProperties.get(KafkaTrafficCaptureSource.MAX_POLL_INTERVAL_KEY)) .map(s->Integer.valueOf((String)s)).orElseThrow(); var executor = Executors.newSingleThreadScheduledExecutor(); executor.schedule(()-> { @@ -155,7 +151,7 @@ public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Excepti } private String renderNextCommitsAsString() { - return "nextCommits="+kafkaSource.nextSetOfCommitsMap.entrySet().stream() + return "nextCommits="+kafkaSource.workingState.nextSetOfCommitsMap.entrySet().stream() .map(kvp->kvp.getKey()+"->"+kvp.getValue()).collect(Collectors.joining(",")); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java similarity index 72% rename from TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java rename to TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java index e87dbab93..103316f7e 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerLongTermTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java @@ -1,41 +1,17 @@ package org.opensearch.migrations.replay.kafka; -import com.google.protobuf.ByteString; -import com.google.protobuf.Timestamp; -import lombok.Lombok; -import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; -import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; -import org.opensearch.migrations.trafficcapture.protos.ReadObservation; -import org.opensearch.migrations.trafficcapture.protos.TrafficObservation; -import org.opensearch.migrations.trafficcapture.protos.TrafficStream; -import org.opensearch.migrations.trafficcapture.protos.TrafficStreamUtils; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; import org.testcontainers.utility.DockerImageName; -import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.Properties; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -43,7 +19,7 @@ @Slf4j @Testcontainers(disabledWithoutDocker = true) @Tag("requiresDocker") -public class KafkaProtobufConsumerLongTermTest { +public class KafkaTrafficCaptureSourceLongTermTest { public static final int TEST_RECORD_COUNT = 10; public static final String TEST_GROUP_CONSUMER_ID = "TEST_GROUP_CONSUMER_ID"; @@ -60,12 +36,12 @@ public class KafkaProtobufConsumerLongTermTest { public void testTrafficCaptureSource() throws Exception { String testTopicName = "TEST_TOPIC"; - var kafkaConsumerProps = KafkaProtobufConsumer.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), + var kafkaConsumerProps = KafkaTrafficCaptureSource.buildKafkaProperties(embeddedKafkaBroker.getBootstrapServers(), TEST_GROUP_CONSUMER_ID, false, null); final long MAX_POLL_MS = 10000; - kafkaConsumerProps.setProperty(KafkaProtobufConsumer.MAX_POLL_INTERVAL_KEY, MAX_POLL_MS+""); + kafkaConsumerProps.setProperty(KafkaTrafficCaptureSource.MAX_POLL_INTERVAL_KEY, MAX_POLL_MS+""); var kafkaConsumer = new KafkaConsumer(kafkaConsumerProps); - var kafkaTrafficCaptureSource = new KafkaProtobufConsumer(kafkaConsumer, testTopicName, + var kafkaTrafficCaptureSource = new KafkaTrafficCaptureSource(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_MS)); var kafkaProducer = KafkaTestUtils.buildKafkaProducer(embeddedKafkaBroker.getBootstrapServers()); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceTest.java similarity index 94% rename from TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerTest.java rename to TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceTest.java index 073cab88c..794da7768 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaProtobufConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceTest.java @@ -25,15 +25,13 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Optional; import java.util.Properties; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; -import java.util.stream.Stream; @Slf4j -class KafkaProtobufConsumerTest { +class KafkaTrafficCaptureSourceTest { public static final int NUM_READ_ITEMS_BOUND = 1000; public static final String TEST_TOPIC_NAME = "TEST_TOPIC_NAME"; @@ -41,7 +39,7 @@ class KafkaProtobufConsumerTest { public void testSupplyTrafficFromSource() { int numTrafficStreams = 10; MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - KafkaProtobufConsumer protobufConsumer = new KafkaProtobufConsumer(mockConsumer, TEST_TOPIC_NAME, + KafkaTrafficCaptureSource protobufConsumer = new KafkaTrafficCaptureSource(mockConsumer, TEST_TOPIC_NAME, Duration.ofHours(1)); initializeMockConsumerTopic(mockConsumer); @@ -82,7 +80,7 @@ public void testSupplyTrafficFromSource() { public void testSupplyTrafficWithUnformattedMessages() { int numTrafficStreams = 10; MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - KafkaProtobufConsumer protobufConsumer = new KafkaProtobufConsumer(mockConsumer, TEST_TOPIC_NAME, + KafkaTrafficCaptureSource protobufConsumer = new KafkaTrafficCaptureSource(mockConsumer, TEST_TOPIC_NAME, Duration.ofHours(1)); initializeMockConsumerTopic(mockConsumer); @@ -132,7 +130,7 @@ public void testSupplyTrafficWithUnformattedMessages() { @Test public void testBuildPropertiesBaseCase() throws IOException { - Properties props = KafkaProtobufConsumer.buildKafkaProperties("brokers", "groupId", false, null); + Properties props = KafkaTrafficCaptureSource.buildKafkaProperties("brokers", "groupId", false, null); Assertions.assertEquals("brokers", props.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); Assertions.assertEquals("org.apache.kafka.common.serialization.StringDeserializer", props.get("key.deserializer")); Assertions.assertEquals("org.apache.kafka.common.serialization.ByteArrayDeserializer", props.get("value.deserializer")); @@ -142,7 +140,7 @@ public void testBuildPropertiesBaseCase() throws IOException { @Test public void testBuildPropertiesMSKAuthEnabled() throws IOException { - Properties props = KafkaProtobufConsumer.buildKafkaProperties("brokers", "groupId", true, null); + Properties props = KafkaTrafficCaptureSource.buildKafkaProperties("brokers", "groupId", true, null); Assertions.assertEquals("brokers", props.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); Assertions.assertEquals("org.apache.kafka.common.serialization.StringDeserializer", props.get("key.deserializer")); Assertions.assertEquals("org.apache.kafka.common.serialization.ByteArrayDeserializer", props.get("value.deserializer")); @@ -157,7 +155,7 @@ public void testBuildPropertiesMSKAuthEnabled() throws IOException { @Test public void testBuildPropertiesWithProvidedPropertyFile() throws IOException { File simplePropertiesFile = new File("src/test/resources/kafka/simple-kafka.properties"); - Properties props = KafkaProtobufConsumer.buildKafkaProperties("brokers", "groupId", true, simplePropertiesFile.getPath()); + Properties props = KafkaTrafficCaptureSource.buildKafkaProperties("brokers", "groupId", true, simplePropertiesFile.getPath()); Assertions.assertEquals("brokers", props.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); Assertions.assertEquals("org.apache.kafka.common.serialization.StringDeserializer", props.get("key.deserializer")); Assertions.assertEquals("org.apache.kafka.common.serialization.ByteArrayDeserializer", props.get("value.deserializer")); From 010e82600c5a01ac30b75ee72608fe790181b85f Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Tue, 21 Nov 2023 00:41:07 -0500 Subject: [PATCH 6/9] Rewrite commit/touch/poll logic to rely upon rebalance events much more. OffsetLifecycleTrackers stay around for the entire duration that a partition is assigned. They're created upon assignment and deleted from the map upon revocation. That partitionToOffsetLifecycleTrackerMap can (& is) now be used to manage the set of partitions that are currently assigned. That's used to pause and resume the consumer so that polls can NOT return any data while we're within touch() (rather than reading and pushing the cursor back). Much of the code can become single threaded within the TrackingKafkaConsumer. Only getNextRequiredTouch can be called from a different thread, so only that has an atomic around it. All other concurrent data structures and locks have gone away. Signed-off-by: Greg Schohn --- .../replay/kafka/KafkaCommitOffsetData.java | 7 - .../kafka/KafkaTrafficCaptureSource.java | 3 +- .../replay/kafka/OffsetLifecycleTracker.java | 4 +- .../kafka/PojoKafkaCommitOffsetData.java | 12 ++ .../replay/kafka/TrackingKafkaConsumer.java | 162 +++++++++--------- .../replay/kafka/KafkaKeepAliveTests.java | 17 +- 6 files changed, 107 insertions(+), 98 deletions(-) create mode 100644 TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/PojoKafkaCommitOffsetData.java diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java index fb8eb4a96..f250ac254 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java @@ -8,11 +8,4 @@ public interface KafkaCommitOffsetData { long getOffset(); int getGeneration(); - @Getter - @AllArgsConstructor - public static class PojoKafkaCommitOffsetData implements KafkaCommitOffsetData { - final int generation; - final int partition; - final long offset; - } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java index 1f99a177e..e99e685c4 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java @@ -209,8 +209,7 @@ public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { throw new IllegalArgumentException("Expected key of type "+TrafficStreamKeyWithKafkaRecordId.class+ " but received "+trafficStreamKey+" (of type="+trafficStreamKey.getClass()+")"); } - var kafkaTsk = (TrafficStreamKeyWithKafkaRecordId) trafficStreamKey; - + workingState.commitKafkaKey((TrafficStreamKeyWithKafkaRecordId) trafficStreamKey); } @Override diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java index 1ec27944f..161b8d03e 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java @@ -9,8 +9,10 @@ class OffsetLifecycleTracker { private final PriorityQueue pQueue = new PriorityQueue<>(); private long cursorHighWatermark; + final int consumerConnectionGeneration; - OffsetLifecycleTracker() { + OffsetLifecycleTracker(int generation) { + this.consumerConnectionGeneration = generation; } boolean isEmpty() { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/PojoKafkaCommitOffsetData.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/PojoKafkaCommitOffsetData.java new file mode 100644 index 000000000..bf3d24a62 --- /dev/null +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/PojoKafkaCommitOffsetData.java @@ -0,0 +1,12 @@ +package org.opensearch.migrations.replay.kafka; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +@Getter +@AllArgsConstructor +public class PojoKafkaCommitOffsetData implements KafkaCommitOffsetData { + final int generation; + final int partition; + final long offset; +} diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java index 0e7ddb6ef..65692bc2f 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java @@ -7,15 +7,16 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.slf4j.event.Level; -//import org.opensearch.migrations.trafficcapture.protos.TrafficStream; import java.time.Clock; import java.time.Duration; import java.time.Instant; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; @@ -28,93 +29,106 @@ public class TrackingKafkaConsumer implements ConsumerRebalanceListener { final String topic; private final Clock clock; - final ConcurrentHashMap partitionToOffsetLifecycleTrackerMap; + /** + * This collection holds the definitive list, as per the rebalance callback, of the partitions + * that are currently assigned to this consumer. The objects are removed when partitions are + * revoked and new objects are only created/inserted when they're assigned. That means that + * the generations of each OffsetLifecycleTracker value may be different. + */ + final Map partitionToOffsetLifecycleTrackerMap; // loosening visibility so that a unit test can read this - final ConcurrentHashMap nextSetOfCommitsMap; + final Map nextSetOfCommitsMap; private final Duration keepAliveInterval; - private final Object offsetLifecycleLock = new Object(); + private final AtomicReference lastTouchTimeRef; private int consumerConnectionGeneration; - private Instant lastPollTimestamp; + private boolean hasPendingCommitsToSend; + private boolean consumerIsPaused; public TrackingKafkaConsumer(Consumer kafkaConsumer, String topic, Duration keepAliveInterval, Clock c) { this.kafkaConsumer = kafkaConsumer; this.topic = topic; this.clock = c; - this.partitionToOffsetLifecycleTrackerMap = new ConcurrentHashMap<>(); - this.nextSetOfCommitsMap = new ConcurrentHashMap<>(); + this.partitionToOffsetLifecycleTrackerMap = new HashMap<>(); + this.nextSetOfCommitsMap = new HashMap<>(); + this.lastTouchTimeRef = new AtomicReference<>(Instant.EPOCH); this.keepAliveInterval = keepAliveInterval; log.error("keepAliveInterval="+keepAliveInterval); } @Override public void onPartitionsRevoked(Collection partitions) { - log.atWarn().setCause(new Exception()).setMessage(()->this+"partitions revoked for "+partitions.stream() + safeCommit(); + partitions.forEach(p->{ + nextSetOfCommitsMap.remove(new TopicPartition(topic, p.partition())); + partitionToOffsetLifecycleTrackerMap.remove(p.partition()); + }); + if (hasPendingCommitsToSend) { + hasPendingCommitsToSend = partitionToOffsetLifecycleTrackerMap.values().stream() + .anyMatch(olt -> !olt.isEmpty()); + } + log.atWarn().setMessage(()->this+"partitions revoked for "+partitions.stream() .map(p->p+"").collect(Collectors.joining(","))).log(); - safeCommitWithRetry(); } @Override public void onPartitionsAssigned(Collection partitions) { - log.atWarn().setCause(new Exception()).setMessage(()->this+"partitions added for "+partitions.stream() + ++consumerConnectionGeneration; + partitions.forEach(p->partitionToOffsetLifecycleTrackerMap.computeIfAbsent(p.partition(), + x->new OffsetLifecycleTracker(consumerConnectionGeneration))); + log.atWarn().setMessage(()->this+"partitions added for "+partitions.stream() .map(p->p+"").collect(Collectors.joining(","))).log(); } public void close() { + log.info("Kafka consumer closing. Committing: " + renderNextCommitsAsString()); kafkaConsumer.close(); - log.info("Kafka consumer closed successfully."); } public Optional getNextRequiredTouch() { - synchronized (offsetLifecycleLock) { - return partitionToOffsetLifecycleTrackerMap.isEmpty() ? Optional.empty() : - Optional.of(lastPollTimestamp.plus(keepAliveInterval)); - } + return hasPendingCommitsToSend ? Optional.of(lastTouchTimeRef.get().plus(keepAliveInterval)) : Optional.empty(); } public void touch() { log.error("TOUCH CALLED"); try { - //kafkaConsumer.pause(); + if (!consumerIsPaused) { + kafkaConsumer.pause(getActivePartitions()); + consumerIsPaused = true; + } var records = kafkaConsumer.poll(Duration.ZERO); - log.atError().setMessage(()->"Polled "+records.count()+" records to keep the consumer alive").log(); - records.forEach(r -> { - try { - var tp = new TopicPartition(r.topic(), r.partition()); - log.atError().setMessage(()->"Resetting "+tp+" to offset="+r.offset()).log(); - kafkaConsumer.seek(tp, r.offset()); - } catch (IllegalStateException e) { - log.atWarn().setCause(e).setMessage(() -> "Caught exception while seeking. " + - "Ignoring so that other records can have their seeks readjusted.").log(); - } - }); + assert records.isEmpty() : "expected no entries once the consumer was paused"; } catch (RuntimeException e) { log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); - resetCommitInfo(); } - safeCommitWithRetry(); - lastPollTimestamp = clock.instant(); + safeCommit(); + lastTouchTimeRef.set(clock.instant()); } - public K createAndTrackKey(int partition, long offset, - Function keyFactory) { - synchronized (offsetLifecycleLock) { - var offsetTracker = partitionToOffsetLifecycleTrackerMap.computeIfAbsent(partition, - p -> new OffsetLifecycleTracker()); - offsetTracker.add(offset); - } - return keyFactory.apply(new KafkaCommitOffsetData.PojoKafkaCommitOffsetData(consumerConnectionGeneration, - partition, offset)); + public K createAndTrackKey(int partition, long offset, Function keyFactory) { + var offsetTracker = partitionToOffsetLifecycleTrackerMap.get(partition); + offsetTracker.add(offset); + return keyFactory.apply(new PojoKafkaCommitOffsetData(consumerConnectionGeneration, partition, offset)); + } + + private Collection getActivePartitions() { + return partitionToOffsetLifecycleTrackerMap.keySet().stream() + .map(p->new TopicPartition(topic,p)).collect(Collectors.toList()); } - ConsumerRecords getNextBatchOfRecords() { - safeCommitWithRetry(); - return safePollWithSwallowedRuntimeExceptions(); + public ConsumerRecords getNextBatchOfRecords() { + if (consumerIsPaused) { + kafkaConsumer.resume(getActivePartitions()); + consumerIsPaused = false; + } + var records = safePollWithSwallowedRuntimeExceptions(); + safeCommit(); + return records; } private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { try { - lastPollTimestamp = clock.instant(); + lastTouchTimeRef.set(clock.instant()); var records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT); log.atLevel(records.isEmpty()? Level.TRACE:Level.INFO) .setMessage(()->"Kafka consumer poll has fetched "+records.count()+" records").log(); @@ -126,7 +140,6 @@ private ConsumerRecords safePollWithSwallowedRuntimeExceptions() } catch (RuntimeException e) { log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); - resetCommitInfo(); return new ConsumerRecords<>(Collections.emptyMap()); } } @@ -141,44 +154,39 @@ void commitKafkaKey(KafkaCommitOffsetData kafkaTsk) { } var p = kafkaTsk.getPartition(); Optional newHeadValue; - synchronized (offsetLifecycleLock) { - var tracker = partitionToOffsetLifecycleTrackerMap.get(p); - newHeadValue = tracker.removeAndReturnNewHead(kafkaTsk); - newHeadValue.ifPresent(o -> { - if (tracker.isEmpty()) { - partitionToOffsetLifecycleTrackerMap.remove(p); - } - nextSetOfCommitsMap.put(new TopicPartition(topic, p), new OffsetAndMetadata(o)); - }); - } + + newHeadValue = partitionToOffsetLifecycleTrackerMap.get(p).removeAndReturnNewHead(kafkaTsk); + newHeadValue.ifPresent(o -> { + hasPendingCommitsToSend = true; + nextSetOfCommitsMap.put(new TopicPartition(topic, p), new OffsetAndMetadata(o)); + }); } - private void safeCommitWithRetry() { - synchronized (offsetLifecycleLock) { - try { - if (!nextSetOfCommitsMap.isEmpty()) { - log.atDebug().setMessage(() -> "Committing " + nextSetOfCommitsMap).log(); - kafkaConsumer.commitSync(nextSetOfCommitsMap); - log.atDebug().setMessage(() -> "Done committing " + nextSetOfCommitsMap).log(); - } + private void safeCommit() { + try { + if (hasPendingCommitsToSend) { + log.atDebug().setMessage(() -> "Committing " + nextSetOfCommitsMap).log(); + kafkaConsumer.commitSync(nextSetOfCommitsMap); + log.atDebug().setMessage(() -> "Done committing " + nextSetOfCommitsMap).log(); nextSetOfCommitsMap.clear(); - } catch (RuntimeException e) { - log.atWarn().setCause(e) - .setMessage(() -> "Error while committing. Purging all commit points since another consumer " + - "may have already begun processing messages BEFORE those commits. Commits being " + - "discarded: " + nextSetOfCommitsMap.entrySet().stream() - .map(kvp -> kvp.getKey() + "->" + kvp.getValue()).collect(Collectors.joining(","))) - .log(); - resetCommitInfo(); } + } catch (RuntimeException e) { + log.atWarn().setCause(e) + .setMessage(() -> "Error while committing. " + + "Another consumer may already be processing messages before these commits. " + + "Commits ARE NOT being discarded here, with the expectation that the revoked callback " + + "(onPartitionsRevoked) will be called. " + + "Within that method, commits for unassigned partitions will be discarded. " + + "After that, touch() or poll() will trigger another commit attempt." + + "Those calls will occur in the near future if assigned partitions have pending commits." + + nextSetOfCommitsMap.entrySet().stream() + .map(kvp -> kvp.getKey() + "->" + kvp.getValue()).collect(Collectors.joining(","))) + .log(); } } - private void resetCommitInfo() { - ++consumerConnectionGeneration; - nextSetOfCommitsMap.clear(); - partitionToOffsetLifecycleTrackerMap.clear(); - + String renderNextCommitsAsString() { + return "nextCommits="+nextSetOfCommitsMap.entrySet().stream() + .map(kvp->kvp.getKey()+"->"+kvp.getValue()).collect(Collectors.joining(",")); } - } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java index 8739dd21f..c32e2ec67 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java @@ -75,7 +75,6 @@ private void setupTestCase() throws Exception { this.keysReceived = new ArrayList(); readNextNStreams(trafficSource, keysReceived, 0, 1); - stopReadsPast(1); KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); } @@ -91,8 +90,7 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { log.warn("Calling commit traffic stream for "+k); trafficSource.commitTrafficStream(k); log.info("finished committing traffic stream"); - stopReadsPast(2); - log.warn("Stop reads past infinity"); + log.warn("Stop reads to infinity"); // this is a way to signal back to the main thread that this thread is done KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); } catch (Exception e) { @@ -111,7 +109,6 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { @Test @Tag("longTest") public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Exception { - trafficSource.stopReadsPast(Instant.MAX); for (int i=0; i<2; ++i) { KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1 + i, sendCompleteCount).get(); } @@ -146,17 +143,15 @@ public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Excepti log.atInfo().setMessage(()->"5 ..."+renderNextCommitsAsString()).log(); Thread.sleep(2*MAX_POLL_INTERVAL_MS); + var keysReceivedUntilDrop3 = keysReceived; + keysReceived = new ArrayList<>(); readNextNStreams(trafficSource, keysReceived, 0, 3); - log.atInfo().setMessage(()->"6 ..."+renderNextCommitsAsString()).log(); + log.atInfo().setMessage(()->"6 ..."+kafkaSource.workingState.renderNextCommitsAsString()).log(); + trafficSource.close(); } private String renderNextCommitsAsString() { - return "nextCommits="+kafkaSource.workingState.nextSetOfCommitsMap.entrySet().stream() - .map(kvp->kvp.getKey()+"->"+kvp.getValue()).collect(Collectors.joining(",")); - } - - private void stopReadsPast(int i) { - trafficSource.stopReadsPast(Instant.EPOCH.plus(Duration.ofDays(i))); + return kafkaSource.workingState.renderNextCommitsAsString(); } @SneakyThrows From 1570a2ba455e0695c75263b97d3dd660ea3fa4d4 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Tue, 21 Nov 2023 11:56:15 -0500 Subject: [PATCH 7/9] Code cleanup in the kafka consumer - Better exception handling, more comments, mild renaming/refactoring. Signed-off-by: Greg Schohn --- .../replay/kafka/KafkaCommitOffsetData.java | 3 - .../kafka/KafkaTrafficCaptureSource.java | 64 ++++++------- .../replay/kafka/OffsetLifecycleTracker.java | 17 ++-- .../replay/kafka/TrackingKafkaConsumer.java | 92 ++++++++++++++----- .../KafkaRestartingTrafficReplayerTest.java | 2 +- .../replay/kafka/KafkaKeepAliveTests.java | 14 ++- ...KafkaTrafficCaptureSourceLongTermTest.java | 2 +- 7 files changed, 122 insertions(+), 72 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java index f250ac254..d110e24c5 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaCommitOffsetData.java @@ -1,8 +1,5 @@ package org.opensearch.migrations.replay.kafka; -import lombok.AllArgsConstructor; -import lombok.Getter; - public interface KafkaCommitOffsetData { int getPartition(); long getOffset(); diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java index e99e685c4..4d6de51df 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSource.java @@ -5,19 +5,15 @@ import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.opensearch.migrations.coreutils.MetricsAttributeKey; import org.opensearch.migrations.coreutils.MetricsEvent; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.common.TopicPartition; import org.opensearch.migrations.coreutils.MetricsLogger; import org.opensearch.migrations.replay.datatypes.ITrafficStreamKey; import org.opensearch.migrations.replay.datatypes.PojoTrafficStreamWithKey; import org.opensearch.migrations.replay.traffic.source.ISimpleTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ITrafficStreamWithKey; import org.opensearch.migrations.trafficcapture.protos.TrafficStream; -import org.slf4j.event.Level; import java.io.FileInputStream; import java.io.IOException; @@ -66,13 +62,14 @@ public class KafkaTrafficCaptureSource implements ISimpleTrafficCaptureSource { public static final String MAX_POLL_INTERVAL_KEY = "max.poll.interval.ms"; + // see https://stackoverflow.com/questions/39730126/difference-between-session-timeout-ms-and-max-poll-interval-ms-for-kafka-0-10 + public static final String DEFAULT_POLL_INTERVAL_MS = "60000"; private static final MetricsLogger metricsLogger = new MetricsLogger("KafkaProtobufConsumer"); - - private final Clock clock; - private final KafkaBehavioralPolicy behavioralPolicy; + + final TrackingKafkaConsumer trackingKafkaConsumer; private final AtomicLong trafficStreamsRead; - final TrackingKafkaConsumer workingState; + private final KafkaBehavioralPolicy behavioralPolicy; public KafkaTrafficCaptureSource(Consumer kafkaConsumer, String topic, Duration keepAliveInterval) { this(kafkaConsumer, topic, keepAliveInterval, Clock.systemUTC(), new KafkaBehavioralPolicy()); @@ -84,11 +81,10 @@ public KafkaTrafficCaptureSource(Consumer kafkaConsumer, Clock clock, @NonNull KafkaBehavioralPolicy behavioralPolicy) { - workingState = new TrackingKafkaConsumer(kafkaConsumer, topic, keepAliveInterval, clock); - this.behavioralPolicy = behavioralPolicy; - kafkaConsumer.subscribe(Collections.singleton(topic), workingState); + trackingKafkaConsumer = new TrackingKafkaConsumer(kafkaConsumer, topic, keepAliveInterval, clock); trafficStreamsRead = new AtomicLong(); - this.clock = clock; + this.behavioralPolicy = behavioralPolicy; + kafkaConsumer.subscribe(Collections.singleton(topic), trackingKafkaConsumer); } public static KafkaTrafficCaptureSource buildKafkaConsumer(@NonNull String brokers, @@ -96,17 +92,25 @@ public static KafkaTrafficCaptureSource buildKafkaConsumer(@NonNull String broke @NonNull String groupId, boolean enableMSKAuth, String propertyFilePath, - Clock clock, - @NonNull KafkaBehavioralPolicy behavioralPolicy) + @NonNull Clock clock, + @NonNull KafkaBehavioralPolicy behavioralPolicy) throws IOException { var kafkaProps = buildKafkaProperties(brokers, groupId, enableMSKAuth, propertyFilePath); - var pollPeriod = Duration.ofMillis(Integer.valueOf((String)kafkaProps.get(MAX_POLL_INTERVAL_KEY))); + kafkaProps.putIfAbsent(MAX_POLL_INTERVAL_KEY, DEFAULT_POLL_INTERVAL_MS); + var pollPeriod = Duration.ofMillis(Long.valueOf((String)kafkaProps.get(MAX_POLL_INTERVAL_KEY))); var keepAlivePeriod = getKeepAlivePeriodFromPollPeriod(pollPeriod); - return new KafkaTrafficCaptureSource(new KafkaConsumer<>(kafkaProps), topic, keepAlivePeriod, - clock, behavioralPolicy); + return new KafkaTrafficCaptureSource(new KafkaConsumer<>(kafkaProps), + topic, keepAlivePeriod, clock, behavioralPolicy); } + /** + * We'll have to 'maintain' touches more frequently than the poll period, otherwise the + * consumer will fall out of the group, putting all the commits in-flight at risk. Notice + * that this doesn't have a bearing on heartbeats, which themselves are maintained through + * Kafka Consumer poll() calls. When those poll calls stop, so does the heartbeat, which + * is more sensitive, but managed via the 'session.timeout.ms' property. + */ private static Duration getKeepAlivePeriodFromPollPeriod(Duration pollPeriod) { return pollPeriod.dividedBy(2); } @@ -142,7 +146,7 @@ public static Properties buildKafkaProperties(@NonNull String brokers, @Override public void touch() { - workingState.touch(); + trackingKafkaConsumer.touch(); } /** @@ -152,7 +156,7 @@ public void touch() { */ @Override public Optional getNextRequiredTouch() { - return workingState.getNextRequiredTouch(); + return trackingKafkaConsumer.getNextRequiredTouch(); } @Override @@ -168,7 +172,7 @@ public CompletableFuture> readNextTrafficStreamChunk public List readNextTrafficStreamSynchronously() { log.atTrace().setMessage("readNextTrafficStreamSynchronously()").log(); try { - var records = workingState.getNextBatchOfRecords(); + var records = trackingKafkaConsumer.getNextBatchOfRecords(); Stream trafficStream = StreamSupport.stream(records.spliterator(), false) .map(kafkaRecord -> { try { @@ -176,20 +180,18 @@ public List readNextTrafficStreamSynchronously() { // Ensure we increment trafficStreamsRead even at a higher log level metricsLogger.atSuccess(MetricsEvent.PARSED_TRAFFIC_STREAM_FROM_KAFKA) .setAttribute(MetricsAttributeKey.CONNECTION_ID, ts.getConnectionId()) - .setAttribute(MetricsAttributeKey.TOPIC_NAME, workingState.topic) + .setAttribute(MetricsAttributeKey.TOPIC_NAME, trackingKafkaConsumer.topic) .setAttribute(MetricsAttributeKey.SIZE_IN_BYTES, ts.getSerializedSize()).emit(); - var key = workingState.createAndTrackKey(kafkaRecord.partition(), kafkaRecord.offset(), + var key = trackingKafkaConsumer.createAndTrackKey(kafkaRecord.partition(), kafkaRecord.offset(), ck -> new TrafficStreamKeyWithKafkaRecordId(ts, ck)); - log.atTrace().setMessage(()->"Parsed traffic stream #{}: {} {}") - .addArgument(trafficStreamsRead.incrementAndGet()) - .addArgument(key) - .addArgument(ts) - .log(); + var trafficStreamsSoFar = trafficStreamsRead.incrementAndGet(); + log.atTrace().setMessage(()->"Parsed traffic stream #" + trafficStreamsSoFar + + ": " + key + " " + ts).log(); return (ITrafficStreamWithKey) new PojoTrafficStreamWithKey(ts, key); } catch (InvalidProtocolBufferException e) { RuntimeException recordError = behavioralPolicy.onInvalidKafkaRecord(kafkaRecord, e); metricsLogger.atError(MetricsEvent.PARSING_TRAFFIC_STREAM_FROM_KAFKA_FAILED, recordError) - .setAttribute(MetricsAttributeKey.TOPIC_NAME, workingState.topic).emit(); + .setAttribute(MetricsAttributeKey.TOPIC_NAME, trackingKafkaConsumer.topic).emit(); if (recordError != null) { throw recordError; } @@ -198,7 +200,7 @@ public List readNextTrafficStreamSynchronously() { }).filter(Objects::nonNull); return trafficStream.collect(Collectors.toList()); } catch (Exception e) { - log.error("Terminating Kafka traffic stream"); + log.atError().setCause(e).setMessage("Terminating Kafka traffic stream due to exception").log(); throw e; } } @@ -209,11 +211,11 @@ public void commitTrafficStream(ITrafficStreamKey trafficStreamKey) { throw new IllegalArgumentException("Expected key of type "+TrafficStreamKeyWithKafkaRecordId.class+ " but received "+trafficStreamKey+" (of type="+trafficStreamKey.getClass()+")"); } - workingState.commitKafkaKey((TrafficStreamKeyWithKafkaRecordId) trafficStreamKey); + trackingKafkaConsumer.commitKafkaKey((TrafficStreamKeyWithKafkaRecordId) trafficStreamKey); } @Override public void close() throws IOException { - workingState.close(); + trackingKafkaConsumer.close(); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java index 161b8d03e..98fc02332 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/OffsetLifecycleTracker.java @@ -5,6 +5,14 @@ import java.util.Optional; import java.util.PriorityQueue; +/** + * This uses a PriorityQueue to find the MINIMUM offset that has yet to be 'committed'. + * This class assumes that add() will be called with ascending offsets and that + * removeAndReturnNewHead may be called in any order. removeAndReturnNewHead returns + * the new commit offset for the partition that this object is associated with. + * It's also assumed that callers MUST call removeAndReturnNewHead for every offset + * that was previously added for commit points to be advanced. + */ @Slf4j class OffsetLifecycleTracker { private final PriorityQueue pQueue = new PriorityQueue<>(); @@ -24,20 +32,17 @@ void add(long offset) { pQueue.add(offset); } - Optional removeAndReturnNewHead(KafkaCommitOffsetData kafkaRecord) { - var offsetToRemove = kafkaRecord.getOffset(); + Optional removeAndReturnNewHead(long offsetToRemove) { var topCursor = pQueue.peek(); var didRemove = pQueue.remove(offsetToRemove); assert didRemove : "Expected all live records to have an entry and for them to be removed only once"; if (topCursor == offsetToRemove) { topCursor = Optional.ofNullable(pQueue.peek()) .orElse(cursorHighWatermark + 1); // most recent cursor was previously popped - log.atDebug().setMessage("Commit called for {}, and new topCursor={}") - .addArgument(offsetToRemove).addArgument(topCursor).log(); + log.atDebug().setMessage("Commit called for " + offsetToRemove + ", and new topCursor=" + topCursor).log(); return Optional.of(topCursor); } else { - log.atDebug().setMessage("Commit called for {}, but topCursor={}") - .addArgument(offsetToRemove).addArgument(topCursor).log(); + log.atDebug().setMessage("Commit called for " + offsetToRemove + ", but topCursor=" + topCursor).log(); return Optional.empty(); } } diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java index 65692bc2f..6379869d0 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/kafka/TrackingKafkaConsumer.java @@ -14,17 +14,32 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; +/** + * This is a wrapper around Kafka's Consumer class that provides tracking of partitions + * and their current (asynchronously 'committed' by the calling contexts) offsets. It + * manages those offsets and the 'active' set of records that have been rendered by this + * consumer, when to pause a poll loop(), and how to deal with consumer rebalances. + */ @Slf4j public class TrackingKafkaConsumer implements ConsumerRebalanceListener { - public static final Duration CONSUMER_POLL_TIMEOUT = Duration.ofSeconds(1); - + /** + * The keep-alive should already be set to a fraction of the max poll timeout for + * the consumer (done outside of this class). The keep-alive tells this class how + * often the caller should be interacting with touch() and poll() calls. As such, + * we want to set up a long enough poll to not overwhelm a broker or client with + * many empty poll() message responses. We also don't want to poll() for so long + * when there aren't messages that there isn't enough time to commit messages, + * which happens after we poll() (on the same thread, as per Consumer requirements). + */ + public static final int POLL_TIMEOUT_KEEP_ALIVE_DIVISOR = 4; private final Consumer kafkaConsumer; final String topic; @@ -42,7 +57,6 @@ public class TrackingKafkaConsumer implements ConsumerRebalanceListener { private final AtomicReference lastTouchTimeRef; private int consumerConnectionGeneration; private boolean hasPendingCommitsToSend; - private boolean consumerIsPaused; public TrackingKafkaConsumer(Consumer kafkaConsumer, String topic, Duration keepAliveInterval, Clock c) { @@ -53,7 +67,6 @@ public TrackingKafkaConsumer(Consumer kafkaConsumer, String topi this.nextSetOfCommitsMap = new HashMap<>(); this.lastTouchTimeRef = new AtomicReference<>(Instant.EPOCH); this.keepAliveInterval = keepAliveInterval; - log.error("keepAliveInterval="+keepAliveInterval); } @Override @@ -80,7 +93,8 @@ public void onPartitionsRevoked(Collection partitions) { } public void close() { - log.info("Kafka consumer closing. Committing: " + renderNextCommitsAsString()); + log.atInfo().setMessage(()->"Kafka consumer closing. " + + "Committing (implicitly by Kafka's consumer): " + nextCommitsToString()).log(); kafkaConsumer.close(); } @@ -89,22 +103,60 @@ public Optional getNextRequiredTouch() { } public void touch() { - log.error("TOUCH CALLED"); + log.trace("touch() called."); + pause(); try { - if (!consumerIsPaused) { - kafkaConsumer.pause(getActivePartitions()); - consumerIsPaused = true; - } var records = kafkaConsumer.poll(Duration.ZERO); - assert records.isEmpty() : "expected no entries once the consumer was paused"; + if (!records.isEmpty()) { + throw new IllegalStateException("Expected no entries once the consumer was paused. " + + "This may have happened because a new assignment slipped into the consumer AFTER pause calls."); + } + } catch (IllegalStateException e) { + throw e; } catch (RuntimeException e) { - log.atWarn().setCause(e).setMessage("Unable to poll the topic: {} with our Kafka consumer. " + - "Swallowing and awaiting next metadata refresh to try again.").addArgument(topic).log(); + log.atWarn().setCause(e).setMessage("Unable to poll the topic: " + topic + " with our Kafka consumer. " + + "Swallowing and awaiting next metadata refresh to try again.").log(); + } finally { + resume(); } safeCommit(); lastTouchTimeRef.set(clock.instant()); } + private void pause() { + var activePartitions = kafkaConsumer.assignment(); + try { + kafkaConsumer.pause(activePartitions); + } catch (IllegalStateException e) { + log.atError().setCause(e).setMessage(()->"Unable to pause the topic partitions: " + topic + ". " + + "The active partitions passed here : " + activePartitions.stream() + .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + "The active partitions as tracked here are: " + getActivePartitions().stream() + .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + "The active partitions according to the consumer: " + kafkaConsumer.assignment().stream() + .map(x->x.toString()).collect(Collectors.joining(",")) + ).log(); + } + } + + private void resume() { + var activePartitions = kafkaConsumer.assignment(); + try { + kafkaConsumer.pause(activePartitions); + } catch (IllegalStateException e) { + log.atError().setCause(e).setMessage(()->"Unable to resume the topic partitions: " + topic + ". " + + "This may not be a fatal error for the entire process as the consumer should eventually" + + " rejoin and rebalance. " + + "The active partitions passed here : " + activePartitions.stream() + .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + "The active partitions as tracked here are: " + getActivePartitions().stream() + .map(x->x.toString()).collect(Collectors.joining(",")) + ". " + + "The active partitions according to the consumer: " + kafkaConsumer.assignment().stream() + .map(x->x.toString()).collect(Collectors.joining(",")) + ).log(); + } + } + public K createAndTrackKey(int partition, long offset, Function keyFactory) { var offsetTracker = partitionToOffsetLifecycleTrackerMap.get(partition); offsetTracker.add(offset); @@ -117,10 +169,6 @@ private Collection getActivePartitions() { } public ConsumerRecords getNextBatchOfRecords() { - if (consumerIsPaused) { - kafkaConsumer.resume(getActivePartitions()); - consumerIsPaused = false; - } var records = safePollWithSwallowedRuntimeExceptions(); safeCommit(); return records; @@ -129,12 +177,12 @@ public ConsumerRecords getNextBatchOfRecords() { private ConsumerRecords safePollWithSwallowedRuntimeExceptions() { try { lastTouchTimeRef.set(clock.instant()); - var records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT); + var records = kafkaConsumer.poll(keepAliveInterval.dividedBy(POLL_TIMEOUT_KEEP_ALIVE_DIVISOR)); log.atLevel(records.isEmpty()? Level.TRACE:Level.INFO) .setMessage(()->"Kafka consumer poll has fetched "+records.count()+" records").log(); - log.atDebug().setMessage(()->"All positions: {"+kafkaConsumer.assignment().stream() + log.atTrace().setMessage(()->"All positions: {"+kafkaConsumer.assignment().stream() .map(tp->tp+": "+kafkaConsumer.position(tp)).collect(Collectors.joining(",")) + "}").log(); - log.atDebug().setMessage(()->"All COMMITTED positions: {"+kafkaConsumer.assignment().stream() + log.atTrace().setMessage(()->"All previously COMMITTED positions: {"+kafkaConsumer.assignment().stream() .map(tp->tp+": "+kafkaConsumer.committed(tp)).collect(Collectors.joining(",")) + "}").log(); return records; } catch (RuntimeException e) { @@ -155,7 +203,7 @@ void commitKafkaKey(KafkaCommitOffsetData kafkaTsk) { var p = kafkaTsk.getPartition(); Optional newHeadValue; - newHeadValue = partitionToOffsetLifecycleTrackerMap.get(p).removeAndReturnNewHead(kafkaTsk); + newHeadValue = partitionToOffsetLifecycleTrackerMap.get(p).removeAndReturnNewHead(kafkaTsk.getOffset()); newHeadValue.ifPresent(o -> { hasPendingCommitsToSend = true; nextSetOfCommitsMap.put(new TopicPartition(topic, p), new OffsetAndMetadata(o)); @@ -185,7 +233,7 @@ private void safeCommit() { } } - String renderNextCommitsAsString() { + String nextCommitsToString() { return "nextCommits="+nextSetOfCommitsMap.entrySet().stream() .map(kvp->kvp.getKey()+"->"+kvp.getValue()).collect(Collectors.joining(",")); } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java index 13562b0c5..8c304af8b 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java @@ -52,7 +52,7 @@ public class KafkaRestartingTrafficReplayerTest { @Container // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility - private KafkaContainer embeddedKafkaBroker = + private final KafkaContainer embeddedKafkaBroker = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); private static class CounterLimitedReceiverFactory implements Supplier> { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java index c32e2ec67..9750cb99f 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java @@ -17,7 +17,6 @@ import org.testcontainers.utility.DockerImageName; import java.time.Duration; -import java.time.Instant; import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -25,7 +24,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; @Slf4j @Testcontainers(disabledWithoutDocker = true) @@ -45,12 +43,12 @@ public class KafkaKeepAliveTests { @Container // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility - private KafkaContainer embeddedKafkaBroker = + private final KafkaContainer embeddedKafkaBroker = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); private KafkaTrafficCaptureSource kafkaSource; /** - * Setup the test case where we've produced and received 1 message, but have not yet committed it. + * Set up the test case where we've produced and received 1 message, but have not yet committed it. * Another message is in the process of being produced. * The BlockingTrafficSource is blocked on everything after a point before the beginning of the test. * @throws Exception @@ -72,7 +70,7 @@ private void setupTestCase() throws Exception { var kafkaConsumer = new KafkaConsumer(kafkaProperties); this.kafkaSource = new KafkaTrafficCaptureSource(kafkaConsumer, testTopicName, Duration.ofMillis(MAX_POLL_INTERVAL_MS)); this.trafficSource = new BlockingTrafficSource(kafkaSource, Duration.ZERO); - this.keysReceived = new ArrayList(); + this.keysReceived = new ArrayList<>(); readNextNStreams(trafficSource, keysReceived, 0, 1); KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 1, sendCompleteCount); @@ -94,7 +92,7 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { // this is a way to signal back to the main thread that this thread is done KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); } catch (Exception e) { - Lombok.sneakyThrow(e); + throw Lombok.sneakyThrow(e); } }, pollIntervalMs, TimeUnit.MILLISECONDS); @@ -146,12 +144,12 @@ public void testBlockedReadsAndBrokenCommitsDontCauseReordering() throws Excepti var keysReceivedUntilDrop3 = keysReceived; keysReceived = new ArrayList<>(); readNextNStreams(trafficSource, keysReceived, 0, 3); - log.atInfo().setMessage(()->"6 ..."+kafkaSource.workingState.renderNextCommitsAsString()).log(); + log.atInfo().setMessage(()->"6 ..."+kafkaSource.trackingKafkaConsumer.nextCommitsToString()).log(); trafficSource.close(); } private String renderNextCommitsAsString() { - return kafkaSource.workingState.renderNextCommitsAsString(); + return kafkaSource.trackingKafkaConsumer.nextCommitsToString(); } @SneakyThrows diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java index 103316f7e..40e15c712 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java @@ -27,7 +27,7 @@ public class KafkaTrafficCaptureSourceLongTermTest { @Container // see https://docs.confluent.io/platform/current/installation/versions-interoperability.html#cp-and-apache-kafka-compatibility - private KafkaContainer embeddedKafkaBroker = + private final KafkaContainer embeddedKafkaBroker = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.5.0")); From e1bc67f3285f9374f8020db1ae5032e67bb7323c Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Wed, 22 Nov 2023 11:56:28 -0500 Subject: [PATCH 8/9] PR Feedback Signed-off-by: Greg Schohn --- .../replay/traffic/source/BlockingTrafficSource.java | 6 +++--- .../replay/traffic/source/ITrafficCaptureSource.java | 5 ++++- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java index 42e106cf0..a84e7b80b 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/BlockingTrafficSource.java @@ -119,8 +119,7 @@ private Void blockIfNeeded() { " lastTimestampSecondsRef="+lastTimestampSecondsRef).log(); while (stopReadingAtRef.get().isBefore(lastTimestampSecondsRef.get())) { try { - log.atInfo().setMessage( - "blocking until signaled to read the next chunk last={} stop={}") + log.atInfo().setMessage("blocking until signaled to read the next chunk last={} stop={}") .addArgument(lastTimestampSecondsRef.get()) .addArgument(stopReadingAtRef.get()) .log(); @@ -131,7 +130,8 @@ private Void blockIfNeeded() { var nextInstant = nextTouchOp.get(); final var nowTime = Instant.now(); var waitIntervalMs = Duration.between(nowTime, nextInstant).toMillis(); - log.error("Next touch at " + nextInstant + " ... in " + waitIntervalMs + "ms (now="+nowTime+")"); + log.atDebug().setMessage(()->"Next touch at " + nextInstant + + " ... in " + waitIntervalMs + "ms (now="+nowTime+")").log(); if (waitIntervalMs <= 0) { underlyingSource.touch(); } else { diff --git a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java index 4e637cfcf..625bde671 100644 --- a/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java +++ b/TrafficCapture/trafficReplayer/src/main/java/org/opensearch/migrations/replay/traffic/source/ITrafficCaptureSource.java @@ -21,10 +21,13 @@ default void close() throws IOException {} /** * Keep-alive call to be used by the BlockingTrafficSource to keep this connection alive if * this is required. + */ + default void touch() {} + + /** * @return The time that the next call to touch() must be completed for this source to stay * active. Empty indicates that touch() does not need to be called to keep the * source active. */ - default void touch() {} default Optional getNextRequiredTouch() { return Optional.empty(); } } From 539b2b139f95a62fe115f1fcbfc75bd4d527b0a7 Mon Sep 17 00:00:00 2001 From: Greg Schohn Date: Wed, 22 Nov 2023 12:08:37 -0500 Subject: [PATCH 9/9] Squash a log verbosities within tests for clarity. Signed-off-by: Greg Schohn --- .../ConditionallyReliableLoggingHttpRequestHandlerTest.java | 2 +- .../opensearch/migrations/replay/JsonTransformerTest.java | 2 +- .../migrations/replay/FullTrafficReplayerTest.java | 6 +++--- .../replay/KafkaRestartingTrafficReplayerTest.java | 4 ++-- ...mpleCapturedTrafficToHttpTransactionAccumulatorTest.java | 2 +- .../datahandlers/http/NettyJsonToByteBufHandlerTest.java | 4 ++-- .../migrations/replay/kafka/KafkaKeepAliveTests.java | 4 ++-- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java b/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java index 531d206ad..4879ee983 100644 --- a/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java +++ b/TrafficCapture/nettyWireLogging/src/test/java/org/opensearch/migrations/trafficcapture/netty/ConditionallyReliableLoggingHttpRequestHandlerTest.java @@ -57,7 +57,7 @@ public CodedOutputStreamAndByteBufferWrapper createStream() { cos.flush(); byteBufferAtomicReference.set(osh.getByteBuffer().flip().asReadOnlyBuffer()); - log.error("byteBufferAtomicReference.get="+byteBufferAtomicReference.get()); + log.trace("byteBufferAtomicReference.get="+byteBufferAtomicReference.get()); return CompletableFuture.completedFuture(flushCount.incrementAndGet()); } diff --git a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/JsonTransformerTest.java b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/JsonTransformerTest.java index a8e911e20..9e36ea7a3 100644 --- a/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/JsonTransformerTest.java +++ b/TrafficCapture/replayerPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/java/org/opensearch/migrations/replay/JsonTransformerTest.java @@ -66,7 +66,7 @@ public void testHttpTransform() throws IOException { .build(); var transformedDocument = transformer.transformJson(documentJson); String transformedJsonOutputStr = emitJson(transformedDocument); - log.error("transformed json document: "+transformedJsonOutputStr); + log.info("transformed json document: "+transformedJsonOutputStr); Assertions.assertTrue(transformedJsonOutputStr.contains(DUMMY_HOSTNAME_TEST_STRING)); } } \ No newline at end of file diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java index a50d4c953..4e0f3765c 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/FullTrafficReplayerTest.java @@ -80,7 +80,7 @@ public void testSingleStreamWithCloseIsCommitted() throws Throwable { TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(0, httpServer.localhostEndpoint(), new IndexWatchingReceiverFactory(), trafficSourceSupplier); Assertions.assertEquals(1, trafficSourceSupplier.nextReadCursor.get()); - log.error("done"); + log.info("done"); } @ParameterizedTest @@ -104,7 +104,7 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { TrafficReplayerRunner.runReplayerUntilSourceWasExhausted(numExpectedRequests, httpServer.localhostEndpoint(), new IndexWatchingReceiverFactory(), trafficSourceSupplier); Assertions.assertEquals(trafficSourceSupplier.streams.size(), trafficSourceSupplier.nextReadCursor.get()); - log.error("done"); + log.info("done"); } @Getter @@ -141,7 +141,7 @@ public ArrayCursorTrafficSourceFactory(List streams) { public ISimpleTrafficCaptureSource get() { var rval = new ArrayCursorTrafficCaptureSource(this); - log.error("trafficSource="+rval+" readCursor="+rval.readCursor.get()+" nextReadCursor="+ nextReadCursor.get()); + log.info("trafficSource="+rval+" readCursor="+rval.readCursor.get()+" nextReadCursor="+ nextReadCursor.get()); return rval; } } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java index 8c304af8b..8d55a6ffc 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/KafkaRestartingTrafficReplayerTest.java @@ -65,7 +65,7 @@ public Consumer get() { var counter = new AtomicInteger(); return tuple -> { if (counter.incrementAndGet() > stopPoint) { - log.error("Request received after our ingest threshold. Throwing. Discarding " + + log.warn("Request received after our ingest threshold. Throwing. Discarding " + tuple.uniqueRequestKey); var nextStopPoint = stopPoint + new Random(stopPoint).nextInt(stopPoint + 1); nextStopPointRef.compareAndSet(stopPoint, nextStopPoint); @@ -99,7 +99,7 @@ public void fullTest(int testSize, boolean randomize) throws Throwable { () -> new SentinelSensingTrafficSource( new KafkaTrafficCaptureSource(buildKafkaConsumer(), TEST_TOPIC_NAME, Duration.ofMillis(DEFAULT_POLL_INTERVAL_MS)))); - log.error("done"); + log.info("done"); } @SneakyThrows diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java index 1ec61749c..53b97d41c 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/SimpleCapturedTrafficToHttpTransactionAccumulatorTest.java @@ -236,7 +236,7 @@ public void onConnectionClose(ISourceTrafficChannelKey key, int channelInteracti static void assertReconstructedTransactionsMatchExpectations(List reconstructedTransactions, int[] expectedRequestSizes, int[] expectedResponseSizes) { - log.error("reconstructedTransactions="+ reconstructedTransactions); + log.debug("reconstructedTransactions="+ reconstructedTransactions); Assertions.assertEquals(expectedRequestSizes.length, reconstructedTransactions.size()); for (int i = 0; i< reconstructedTransactions.size(); ++i) { Assertions.assertEquals((long) expectedRequestSizes[i], diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonToByteBufHandlerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonToByteBufHandlerTest.java index 818a53988..9da438ce4 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonToByteBufHandlerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/NettyJsonToByteBufHandlerTest.java @@ -34,7 +34,7 @@ public class NettyJsonToByteBufHandlerTest { @Test public void testThatHttpContentsAreRepackagedToChunkSizeSpec() { for (int i=0; i<10; ++i) { - log.error("Testing w/ random seed="+i); + log.info("Testing w/ random seed="+i); testWithSeed(new Random(i)); System.gc(); System.runFinalization(); @@ -90,7 +90,7 @@ private static List getByteBufSizesFromChannel(EmbeddedChannel channel) static byte nonce = 0; private int writeAndCheck(EmbeddedChannel channel, ArrayList sizesWrittenList, int len) { var bytes = new byte[len]; - log.warn("Writing "+len); + log.debug("Writing "+len); sizesWrittenList.add(len); Arrays.fill(bytes, nonce++); var httpContent = new DefaultHttpContent(Unpooled.wrappedBuffer(bytes)); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java index 9750cb99f..b2319765a 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaKeepAliveTests.java @@ -85,10 +85,10 @@ public void testTimeoutsDontOccurForSlowPolls() throws Exception { executor.schedule(()-> { try { var k = keysReceived.get(0); - log.warn("Calling commit traffic stream for "+k); + log.info("Calling commit traffic stream for "+k); trafficSource.commitTrafficStream(k); log.info("finished committing traffic stream"); - log.warn("Stop reads to infinity"); + log.info("Stop reads to infinity"); // this is a way to signal back to the main thread that this thread is done KafkaTestUtils.produceKafkaRecord(testTopicName, kafkaProducer, 2, sendCompleteCount); } catch (Exception e) {