From 4c393332ac179d055d2e06a8f34170ed3d4ae628 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Fri, 19 Jun 2020 16:26:17 -0500 Subject: [PATCH 01/23] KAFKA-10173: Directly use Arrays.equals for version comparison --- .../InMemoryTimeOrderedKeyValueBuffer.java | 39 ++++++++----------- 1 file changed, 16 insertions(+), 23 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 9feccb9d57a1e..d3fb8dfb97eff 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -41,6 +41,7 @@ import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -58,10 +59,12 @@ public final class InMemoryTimeOrderedKeyValueBuffer implements TimeOrderedKeyValueBuffer { private static final BytesSerializer KEY_SERIALIZER = new BytesSerializer(); private static final ByteArraySerializer VALUE_SERIALIZER = new ByteArraySerializer(); + private static final byte[] V_1_CHANGELOG_HEADER_VALUE = {(byte) 1}; private static final RecordHeaders V_1_CHANGELOG_HEADERS = - new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 1})}); + new RecordHeaders(new Header[] {new RecordHeader("v", V_1_CHANGELOG_HEADER_VALUE)}); + private static final byte[] V_2_CHANGELOG_HEADER_VALUE = {(byte) 2}; private static final RecordHeaders V_2_CHANGELOG_HEADERS = - new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 2})}); + new RecordHeaders(new Header[] {new RecordHeader("v", V_2_CHANGELOG_HEADER_VALUE)}); private static final String METRIC_SCOPE = "in-memory-suppression"; private final Map index = new HashMap<>(); @@ -286,6 +289,15 @@ private void logTombstone(final Bytes key) { private void restoreBatch(final Collection> batch) { for (final ConsumerRecord record : batch) { + if (record.partition() != partition) { + throw new IllegalStateException( + String.format( + "record partition [%d] is being restored by the wrong suppress partition [%d]", + record.partition(), + partition + ) + ); + } final Bytes key = Bytes.wrap(record.key()); if (record.value() == null) { // This was a tombstone. Delete the record. @@ -299,16 +311,6 @@ private void restoreBatch(final Collection> batch minTimestamp = sortedMap.isEmpty() ? Long.MAX_VALUE : sortedMap.firstKey().time(); } } - - if (record.partition() != partition) { - throw new IllegalStateException( - String.format( - "record partition [%d] is being restored by the wrong suppress partition [%d]", - record.partition(), - partition - ) - ); - } } else { if (record.headers().lastHeader("v") == null) { // in this case, the changelog value is just the serialized record value @@ -339,7 +341,7 @@ private void restoreBatch(final Collection> batch recordContext ) ); - } else if (V_1_CHANGELOG_HEADERS.lastHeader("v").equals(record.headers().lastHeader("v"))) { + } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_1_CHANGELOG_HEADER_VALUE)) { // in this case, the changelog value is a serialized ContextualRecord final ByteBuffer timeAndValue = ByteBuffer.wrap(record.value()); final long time = timeAndValue.getLong(); @@ -361,7 +363,7 @@ private void restoreBatch(final Collection> batch contextualRecord.recordContext() ) ); - } else if (V_2_CHANGELOG_HEADERS.lastHeader("v").equals(record.headers().lastHeader("v"))) { + } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_2_CHANGELOG_HEADER_VALUE)) { // in this case, the changelog value is a serialized BufferValue final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); @@ -372,15 +374,6 @@ private void restoreBatch(final Collection> batch throw new IllegalArgumentException("Restoring apparently invalid changelog record: " + record); } } - if (record.partition() != partition) { - throw new IllegalStateException( - String.format( - "record partition [%d] is being restored by the wrong suppress partition [%d]", - record.partition(), - partition - ) - ); - } } updateBufferMetrics(); } From 98786b338b41173a5232754d5d83c0f19392b908 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 10:25:01 -0500 Subject: [PATCH 02/23] converting upgrade test to smoke test --- .../kafka/streams/tests/SmokeTestUtil.java | 2 +- .../kafka/streams/tests/SmokeTestClient.java | 305 +++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 622 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 134 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 99 +++ .../kafka/streams/tests/SmokeTestClient.java | 305 +++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 622 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 134 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 99 +++ tests/kafkatest/services/streams.py | 36 +- .../tests/streams/streams_upgrade_test.py | 142 ++-- 11 files changed, 2428 insertions(+), 72 deletions(-) create mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index 990bfdb73fede..e8ec04c23ca52 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -75,7 +75,7 @@ public void process(final Object key, final Object value) { @Override public void close() { - System.out.printf("Close processor for task %s", context().taskId()); + System.out.printf("Close processor for task %s%n", context().taskId()); System.out.println("processed " + numRecordsProcessed + " records"); final long processed; if (largestOffset >= smallestOffset) { diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000000..77458877bc51a --- /dev/null +++ b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler((t, e) -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + e.printStackTrace(System.out); + uncaughtException = true; + streams.close(Duration.ofSeconds(30)); + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); + fullProps.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); + fullProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); + fullProps.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + fullProps.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); + fullProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + fullProps.put(ProducerConfig.ACKS_CONFIG, "all"); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) + .reduce((l, r) -> l + r); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000000..ac83cd95ebaaf --- /dev/null +++ b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,622 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + List> needRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, needRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor("data"); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + )); + } + } + return Collections.unmodifiableMap(allData); + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000000..e8ec04c23ca52 --- /dev/null +++ b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.time.Instant; + +public class SmokeTestUtil { + + final static int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return new ProcessorSupplier() { + @Override + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Object key, final Object value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + if (smallestOffset > context().offset()) { + smallestOffset = context().offset(); + } + if (largestOffset < context().offset()) { + largestOffset = context().offset(); + } + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000000..07c7d5d82ebff --- /dev/null +++ b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + System.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + System.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE); + + System.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000000..77458877bc51a --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler((t, e) -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + e.printStackTrace(System.out); + uncaughtException = true; + streams.close(Duration.ofSeconds(30)); + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); + fullProps.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); + fullProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); + fullProps.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + fullProps.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); + fullProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + fullProps.put(ProducerConfig.ACKS_CONFIG, "all"); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) + .reduce((l, r) -> l + r); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000000..ac83cd95ebaaf --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,622 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + List> needRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, needRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor("data"); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + )); + } + } + return Collections.unmodifiableMap(allData); + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000000..e8ec04c23ca52 --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.time.Instant; + +public class SmokeTestUtil { + + final static int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return new ProcessorSupplier() { + @Override + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Object key, final Object value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + if (smallestOffset > context().offset()) { + smallestOffset = context().offset(); + } + if (largestOffset < context().offset()) { + largestOffset = context().offset(); + } + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000000..07c7d5d82ebff --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + System.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + System.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE); + + System.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 72847f6328722..890b9c87b93a1 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -312,6 +312,15 @@ def __init__(self, test_context, kafka, command, processing_guarantee = 'at_leas command) self.NUM_THREADS = num_threads self.PROCESSING_GUARANTEE = processing_guarantee + self.KAFKA_STREAMS_VERSION = "" + self.UPGRADE_FROM = None + + def set_version(self, kafka_streams_version): + self.KAFKA_STREAMS_VERSION = kafka_streams_version + + def set_upgrade_from(self, upgrade_from): + self.UPGRADE_FROM = upgrade_from + def prop_file(self): properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT, @@ -319,9 +328,33 @@ def prop_file(self): streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE, streams_property.NUM_THREADS: self.NUM_THREADS} + if self.UPGRADE_FROM is not None: + properties['upgrade.from'] = self.UPGRADE_FROM + cfg = KafkaConfig(**properties) return cfg.render() + def start_cmd(self, node): + args = self.args.copy() + args['config_file'] = self.CONFIG_FILE + args['stdout'] = self.STDOUT_FILE + args['stderr'] = self.STDERR_FILE + args['pidfile'] = self.PID_FILE + args['log4j'] = self.LOG4J_CONFIG_FILE + args['version'] = self.KAFKA_STREAMS_VERSION + args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) + + cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\";" \ + " INCLUDE_TEST_JARS=true UPGRADE_KAFKA_STREAMS_TEST_VERSION=%(version)s" \ + " bash -x %(kafka_run_class)s %(streams_class_name)s" \ + " %(config_file)s %(user_test_args1)s" \ + " & echo $! >&3 ) " \ + "1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args + + self.logger.info("Executing streams cmd: " + cmd) + + return cmd + class StreamsEosTestBaseService(StreamsTestBaseService): """Base class for Streams EOS Test services providing some common settings and functionality""" @@ -370,6 +403,8 @@ def start_cmd(self, node): " %(config_file)s %(user_test_args1)s %(disable_auto_terminate)s" \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args + self.logger.info("Executing streams cmd: " + cmd) + return cmd class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): @@ -380,7 +415,6 @@ class StreamsEosTestDriverService(StreamsEosTestBaseService): def __init__(self, test_context, kafka): super(StreamsEosTestDriverService, self).__init__(test_context, kafka, "not-required", "run") - class StreamsEosTestJobRunnerService(StreamsEosTestBaseService): def __init__(self, test_context, kafka, processing_guarantee): super(StreamsEosTestJobRunnerService, self).__init__(test_context, kafka, processing_guarantee, "process") diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 0f0719a3ef8c9..3daf0dd01e947 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -189,15 +189,14 @@ def test_upgrade_downgrade_brokers(self, from_version, to_version): processor.stop() processor.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % processor.STDOUT_FILE, allow_fail=False) - @matrix(from_version=metadata_2_versions, to_version=metadata_2_versions) - def test_simple_upgrade_downgrade(self, from_version, to_version): + @matrix(from_version=metadata_1_versions, to_version=backward_compatible_metadata_2_versions) + @matrix(from_version=metadata_1_versions, to_version=metadata_3_or_higher_versions) + @matrix(from_version=metadata_2_versions, to_version=metadata_3_or_higher_versions) + def test_metadata_upgrade(self, from_version, to_version): """ - Starts 3 KafkaStreams instances with , and upgrades one-by-one to + Starts 3 KafkaStreams instances with version and upgrades one-by-one to """ - if from_version == to_version: - return - self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() @@ -218,10 +217,16 @@ def test_simple_upgrade_downgrade(self, from_version, to_version): counter = 1 random.seed() - # upgrade one-by-one via rolling bounce + # first rolling bounce random.shuffle(self.processors) for p in self.processors: p.CLEAN_NODE_ENABLED = False + self.do_stop_start_bounce(p, from_version[:-2], to_version, counter) + counter = counter + 1 + + # second rolling bounce + random.shuffle(self.processors) + for p in self.processors: self.do_stop_start_bounce(p, None, to_version, counter) counter = counter + 1 @@ -237,25 +242,42 @@ def test_simple_upgrade_downgrade(self, from_version, to_version): timeout_sec=60, err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) - @matrix(from_version=metadata_1_versions, to_version=backward_compatible_metadata_2_versions) - @matrix(from_version=metadata_1_versions, to_version=metadata_3_or_higher_versions) - @matrix(from_version=metadata_2_versions, to_version=metadata_3_or_higher_versions) - def test_metadata_upgrade(self, from_version, to_version): + @matrix(from_version=metadata_2_versions, to_version=metadata_2_versions) + @matrix(from_version=metadata_3_or_higher_versions, to_version=metadata_3_or_higher_versions) + def test_simple_upgrade_downgrade(self, from_version, to_version): """ - Starts 3 KafkaStreams instances with version and upgrades one-by-one to + Starts 3 KafkaStreams instances with , and upgrades one-by-one to """ + if from_version == to_version: + return + self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() - self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics=self.topics) + self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, topics={ + 'echo' : { 'partitions': 5, 'replication-factor': 1 }, + 'data' : { 'partitions': 5, 'replication-factor': 1 }, + 'min' : { 'partitions': 5, 'replication-factor': 1 }, + 'min-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, + 'min-raw' : { 'partitions': 5, 'replication-factor': 1 }, + 'max' : { 'partitions': 5, 'replication-factor': 1 }, + 'sum' : { 'partitions': 5, 'replication-factor': 1 }, + 'sws-raw' : { 'partitions': 5, 'replication-factor': 1 }, + 'sws-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, + 'dif' : { 'partitions': 5, 'replication-factor': 1 }, + 'cnt' : { 'partitions': 5, 'replication-factor': 1 }, + 'avg' : { 'partitions': 5, 'replication-factor': 1 }, + 'wcnt' : { 'partitions': 5, 'replication-factor': 1 }, + 'tagg' : { 'partitions': 5, 'replication-factor': 1 } + }) self.kafka.start() self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.driver.disable_auto_terminate() - self.processor1 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) - self.processor2 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) - self.processor3 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once") + self.processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once") + self.processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once") self.driver.start() self.start_all_nodes_with(from_version) @@ -265,16 +287,10 @@ def test_metadata_upgrade(self, from_version, to_version): counter = 1 random.seed() - # first rolling bounce + # upgrade one-by-one via rolling bounce random.shuffle(self.processors) for p in self.processors: p.CLEAN_NODE_ENABLED = False - self.do_stop_start_bounce(p, from_version[:-2], to_version, counter) - counter = counter + 1 - - # second rolling bounce - random.shuffle(self.processors) - for p in self.processors: self.do_stop_start_bounce(p, None, to_version, counter) counter = counter + 1 @@ -349,56 +365,42 @@ def get_version_string(self, version): def start_all_nodes_with(self, version): kafka_version_str = self.get_version_string(version) - # start first with self.prepare_for(self.processor1, version) - node1 = self.processor1.node - with node1.account.monitor_log(self.processor1.STDOUT_FILE) as monitor: - with node1.account.monitor_log(self.processor1.LOG_FILE) as log_monitor: - self.processor1.start() - log_monitor.wait_until(kafka_version_str, - timeout_sec=60, - err_msg="Could not detect Kafka Streams version " + version + " " + str(node1.account)) - monitor.wait_until(self.processed_msg, - timeout_sec=60, - err_msg="Never saw output '%s' on " % self.processed_msg + str(node1.account)) - - # start second with self.prepare_for(self.processor2, version) - node2 = self.processor2.node - with node1.account.monitor_log(self.processor1.STDOUT_FILE) as first_monitor: - with node2.account.monitor_log(self.processor2.STDOUT_FILE) as second_monitor: - with node2.account.monitor_log(self.processor2.LOG_FILE) as log_monitor: - self.processor2.start() - log_monitor.wait_until(kafka_version_str, - timeout_sec=60, - err_msg="Could not detect Kafka Streams version " + version + " on " + str(node2.account)) - first_monitor.wait_until(self.processed_msg, - timeout_sec=60, - err_msg="Never saw output '%s' on " % self.processed_msg + str(node1.account)) - second_monitor.wait_until(self.processed_msg, - timeout_sec=60, - err_msg="Never saw output '%s' on " % self.processed_msg + str(node2.account)) - - # start third with self.prepare_for(self.processor3, version) - node3 = self.processor3.node - with node1.account.monitor_log(self.processor1.STDOUT_FILE) as first_monitor: - with node2.account.monitor_log(self.processor2.STDOUT_FILE) as second_monitor: - with node3.account.monitor_log(self.processor3.STDOUT_FILE) as third_monitor: - with node3.account.monitor_log(self.processor3.LOG_FILE) as log_monitor: - self.processor3.start() - log_monitor.wait_until(kafka_version_str, - timeout_sec=60, - err_msg="Could not detect Kafka Streams version " + version + " on " + str(node3.account)) - first_monitor.wait_until(self.processed_msg, - timeout_sec=60, - err_msg="Never saw output '%s' on " % self.processed_msg + str(node1.account)) - second_monitor.wait_until(self.processed_msg, - timeout_sec=60, - err_msg="Never saw output '%s' on " % self.processed_msg + str(node2.account)) - third_monitor.wait_until(self.processed_msg, - timeout_sec=60, - err_msg="Never saw output '%s' on " % self.processed_msg + str(node3.account)) + + self.processor1.start() + self.processor2.start() + self.processor3.start() + + # double-check the version + self.wait_for_verification(self.processor1, kafka_version_str, self.processor1.LOG_FILE) + self.wait_for_verification(self.processor2, kafka_version_str, self.processor1.LOG_FILE) + self.wait_for_verification(self.processor3, kafka_version_str, self.processor1.LOG_FILE) + + # wait for the members to join + self.wait_for_verification(self.processor1, "SMOKE-TEST-CLIENT-STARTED", self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor2, "SMOKE-TEST-CLIENT-STARTED", self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor3, "SMOKE-TEST-CLIENT-STARTED", self.processor1.STDOUT_FILE) + + # make sure they've processed something + self.wait_for_verification(self.processor1, self.processed_msg, self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor2, self.processed_msg, self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor3, self.processed_msg, self.processor1.STDOUT_FILE) + + def wait_for_verification(self, processor, message, file, num_lines=1): + wait_until(lambda: self.verify_from_file(processor, message, file) >= num_lines, + timeout_sec=60, + err_msg="Did expect to read '%s' from %s" % (message, processor.node.account)) + + @staticmethod + def verify_from_file(processor, message, file): + result = processor.node.account.ssh_output("grep -E '%s' %s | wc -l" % (message, file), allow_fail=False) + try: + return int(result) + except ValueError: + self.logger.warn("Command failed with ValueError: " + result) + return 0 @staticmethod def prepare_for(processor, version): From 5ba0c59bd7706f18ef8e9604f699827e8f7e5f29 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 10:25:17 -0500 Subject: [PATCH 03/23] wip debugging suppress buffer --- .../streams/state/internals/BufferValue.java | 52 ++++++++++++++++++- .../InMemoryTimeOrderedKeyValueBuffer.java | 3 +- 2 files changed, 52 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java index b52ec24d20cc1..7b79f396fe547 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java @@ -16,12 +16,16 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.kstream.internals.FullChangeSerde; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Objects; +import static java.util.Objects.requireNonNull; + public final class BufferValue { private static final int NULL_VALUE_SENTINEL = -1; private static final int OLD_PREV_DUPLICATE_VALUE_SENTINEL = -2; @@ -30,6 +34,50 @@ public final class BufferValue { private final byte[] newValue; private final ProcessorRecordContext recordContext; + public static void main(String[] args) { + final String str="00000172D1434655000000000000006F0000000C6C6F67732D696E6772657373000000180000000000000059FFFFFFFF00000051080110AC051A2436346139616437662D313838352D346234342D613163302D633134346565633162636665222436346139616437662D313838352D346234342D613163302D633134346565633162636665FFFFFFFF00000172D1434655"; + final byte[] arr = hexStringToByteArray(str); + + // in this case, the changelog value is a serialized ContextualRecord + final ByteBuffer timeAndValue = ByteBuffer.wrap(arr); + final long time = timeAndValue.getLong(); + final byte[] changelogValue = new byte[arr.length - 8]; + timeAndValue.get(changelogValue); + + final ContextualRecord contextualRecord = ContextualRecord.deserialize(ByteBuffer.wrap(changelogValue)); + final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); + + System.out.println(change); + + final BufferValue deserialize = deserialize(ByteBuffer.wrap(arr)); + System.out.println(deserialize); + } + + public static byte[] hexStringToByteArray(String s) { + int len = s.length(); + byte[] data = new byte[len / 2]; + for (int i = 0; i < len; i += 2) { + data[i / 2] = (byte) ((Character.digit(s.charAt(i), 16) << 4) + + Character.digit(s.charAt(i+1), 16)); + } + return data; + } + + private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); + + private static String bytesToHex(byte[] bytes) { + if (bytes == null) { + return "null"; + } + char[] hexChars = new char[bytes.length * 2]; + for (int j = 0; j < bytes.length; j++) { + int v = bytes[j] & 0xFF; + hexChars[j * 2] = HEX_ARRAY[v >>> 4]; + hexChars[j * 2 + 1] = HEX_ARRAY[v & 0x0F]; + } + return new String(hexChars); + } + BufferValue(final byte[] priorValue, final byte[] oldValue, final byte[] newValue, @@ -80,6 +128,8 @@ static BufferValue deserialize(final ByteBuffer buffer) { buffer.get(oldValue); } + System.out.println("prior value: " + bytesToHex(priorValue)); + final byte[] newValue = extractValue(buffer); return new BufferValue(priorValue, oldValue, newValue, context); @@ -120,7 +170,7 @@ ByteBuffer serialize(final int endPadding) { if (oldValue == null) { buffer.putInt(NULL_VALUE_SENTINEL); - } else if (priorValue == oldValue) { + } else if (Arrays.equals(priorValue, oldValue)) { buffer.putInt(OLD_PREV_DUPLICATE_VALUE_SENTINEL); } else { buffer.putInt(sizeOfOldValue); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index d3fb8dfb97eff..5a7b944fbf09e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -474,8 +474,7 @@ public void put(final long time, final BufferValue buffered = getBuffered(serializedKey); final byte[] serializedPriorValue; if (buffered == null) { - final V priorValue = value.oldValue; - serializedPriorValue = (priorValue == null) ? null : valueSerde.innerSerde().serializer().serialize(changelogTopic, priorValue); + serializedPriorValue = serialChange.oldValue; } else { serializedPriorValue = buffered.priorValue(); } From f556a41f6ad96b4c2738df801196a2e5c3828a98 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 17:15:09 -0500 Subject: [PATCH 04/23] asdf --- bin/kafka-run-class.sh | 1 + .../kafka/streams/state/internals/BufferValue.java | 10 ++++------ .../internals/InMemoryTimeOrderedKeyValueBuffer.java | 11 +++++++++-- tests/kafkatest/services/streams.py | 2 +- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 95f1a3b606633..92d13413f6b92 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -311,6 +311,7 @@ CLASSPATH=${CLASSPATH#:} # If Cygwin is detected, classpath is converted to Windows format. (( CYGWIN )) && CLASSPATH=$(cygpath --path --mixed "${CLASSPATH}") +echo "CLASSPATH=${CLASSPATH}" # Launch mode if [ "x$DAEMON_MODE" = "xtrue" ]; then nohup "$JAVA" $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp "$CLASSPATH" $KAFKA_OPTS "$@" > "$CONSOLE_OUTPUT_FILE" 2>&1 < /dev/null & diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java index 7b79f396fe547..b2ca9733cec03 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java @@ -65,7 +65,7 @@ public static byte[] hexStringToByteArray(String s) { private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); - private static String bytesToHex(byte[] bytes) { + public static String bytesToHex(byte[] bytes) { if (bytes == null) { return "null"; } @@ -128,8 +128,6 @@ static BufferValue deserialize(final ByteBuffer buffer) { buffer.get(oldValue); } - System.out.println("prior value: " + bytesToHex(priorValue)); - final byte[] newValue = extractValue(buffer); return new BufferValue(priorValue, oldValue, newValue, context); @@ -221,9 +219,9 @@ public int hashCode() { @Override public String toString() { return "BufferValue{" + - "priorValue=" + Arrays.toString(priorValue) + - ", oldValue=" + Arrays.toString(oldValue) + - ", newValue=" + Arrays.toString(newValue) + + "priorValue=0x" + bytesToHex(priorValue) + + ", oldValue=0x" + bytesToHex(oldValue) + + ", newValue=0x" + bytesToHex(newValue) + ", recordContext=" + recordContext + '}'; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 5a7b944fbf09e..7b4253631a9c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -261,11 +261,15 @@ private void logValue(final Bytes key, final BufferKey bufferKey, final BufferVa final int sizeOfBufferTime = Long.BYTES; final ByteBuffer buffer = value.serialize(sizeOfBufferTime); buffer.putLong(bufferKey.time()); - + final byte[] array = buffer.array(); + System.err.printf("Serializing v2:"); + System.err.printf(" key: " + BufferValue.bytesToHex(key.get())); + System.err.printf(" value: " + value); + System.err.printf(" serialized: " + BufferValue.bytesToHex(array)); ((RecordCollector.Supplier) context).recordCollector().send( changelogTopic, key, - buffer.array(), + array, V_2_CHANGELOG_HEADERS, partition, null, @@ -365,6 +369,9 @@ private void restoreBatch(final Collection> batch ); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_2_CHANGELOG_HEADER_VALUE)) { // in this case, the changelog value is a serialized BufferValue + System.err.println("Deserializing " + record); + System.err.println(" key: " + BufferValue.bytesToHex(record.key())); + System.err.println(" value: " + BufferValue.bytesToHex(record.value())); final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 890b9c87b93a1..a913b1c65201e 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -346,7 +346,7 @@ def start_cmd(self, node): cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\";" \ " INCLUDE_TEST_JARS=true UPGRADE_KAFKA_STREAMS_TEST_VERSION=%(version)s" \ - " bash -x %(kafka_run_class)s %(streams_class_name)s" \ + " %(kafka_run_class)s %(streams_class_name)s" \ " %(config_file)s %(user_test_args1)s" \ " & echo $! >&3 ) " \ "1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args From 7af7935a67c75ee263d05187a765bb53ed6030d7 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 17:47:46 -0500 Subject: [PATCH 05/23] fix --- .../state/internals/InMemoryTimeOrderedKeyValueBuffer.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 7b4253631a9c9..ff86ffea20d5a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -60,8 +60,6 @@ public final class InMemoryTimeOrderedKeyValueBuffer implements TimeOrdere private static final BytesSerializer KEY_SERIALIZER = new BytesSerializer(); private static final ByteArraySerializer VALUE_SERIALIZER = new ByteArraySerializer(); private static final byte[] V_1_CHANGELOG_HEADER_VALUE = {(byte) 1}; - private static final RecordHeaders V_1_CHANGELOG_HEADERS = - new RecordHeaders(new Header[] {new RecordHeader("v", V_1_CHANGELOG_HEADER_VALUE)}); private static final byte[] V_2_CHANGELOG_HEADER_VALUE = {(byte) 2}; private static final RecordHeaders V_2_CHANGELOG_HEADERS = new RecordHeaders(new Header[] {new RecordHeader("v", V_2_CHANGELOG_HEADER_VALUE)}); From 495aebc9609b09318f246cbe801e55662263d14e Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 22:29:42 -0500 Subject: [PATCH 06/23] upgrade test passed (2.3.1 -> trunk) --- .../streams/state/internals/BufferValue.java | 13 +++--- .../InMemoryTimeOrderedKeyValueBuffer.java | 40 +++++++++++++++---- .../tests/streams/streams_upgrade_test.py | 4 +- 3 files changed, 42 insertions(+), 15 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java index b2ca9733cec03..9ace9368dd2ab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java @@ -69,8 +69,11 @@ public static String bytesToHex(byte[] bytes) { if (bytes == null) { return "null"; } - char[] hexChars = new char[bytes.length * 2]; - for (int j = 0; j < bytes.length; j++) { + final int length = bytes.length; + char[] hexChars = new char[2 + length * 2]; + hexChars[0] = '0'; + hexChars[1] = 'x'; + for (int j = 2; j < length; j++) { int v = bytes[j] & 0xFF; hexChars[j * 2] = HEX_ARRAY[v >>> 4]; hexChars[j * 2 + 1] = HEX_ARRAY[v & 0x0F]; @@ -219,9 +222,9 @@ public int hashCode() { @Override public String toString() { return "BufferValue{" + - "priorValue=0x" + bytesToHex(priorValue) + - ", oldValue=0x" + bytesToHex(oldValue) + - ", newValue=0x" + bytesToHex(newValue) + + "priorValue=" + bytesToHex(priorValue) + + ", oldValue=" + bytesToHex(oldValue) + + ", newValue=" + bytesToHex(newValue) + ", recordContext=" + recordContext + '}'; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index ff86ffea20d5a..ff71da6543b64 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -61,8 +61,9 @@ public final class InMemoryTimeOrderedKeyValueBuffer implements TimeOrdere private static final ByteArraySerializer VALUE_SERIALIZER = new ByteArraySerializer(); private static final byte[] V_1_CHANGELOG_HEADER_VALUE = {(byte) 1}; private static final byte[] V_2_CHANGELOG_HEADER_VALUE = {(byte) 2}; - private static final RecordHeaders V_2_CHANGELOG_HEADERS = - new RecordHeaders(new Header[] {new RecordHeader("v", V_2_CHANGELOG_HEADER_VALUE)}); + private static final byte[] V_3_CHANGELOG_HEADER_VALUE = {(byte) 3}; + private static final RecordHeaders CHANGELOG_HEADERS = + new RecordHeaders(new Header[] {new RecordHeader("v", V_3_CHANGELOG_HEADER_VALUE)}); private static final String METRIC_SCOPE = "in-memory-suppression"; private final Map index = new HashMap<>(); @@ -260,15 +261,15 @@ private void logValue(final Bytes key, final BufferKey bufferKey, final BufferVa final ByteBuffer buffer = value.serialize(sizeOfBufferTime); buffer.putLong(bufferKey.time()); final byte[] array = buffer.array(); - System.err.printf("Serializing v2:"); - System.err.printf(" key: " + BufferValue.bytesToHex(key.get())); - System.err.printf(" value: " + value); - System.err.printf(" serialized: " + BufferValue.bytesToHex(array)); + System.err.println("Serializing v3:"); + System.err.println(" key: " + BufferValue.bytesToHex(key.get())); + System.err.println(" value: " + value); + System.err.println(" serialized: " + BufferValue.bytesToHex(array)); ((RecordCollector.Supplier) context).recordCollector().send( changelogTopic, key, array, - V_2_CHANGELOG_HEADERS, + CHANGELOG_HEADERS, partition, null, KEY_SERIALIZER, @@ -367,12 +368,35 @@ private void restoreBatch(final Collection> batch ); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_2_CHANGELOG_HEADER_VALUE)) { // in this case, the changelog value is a serialized BufferValue - System.err.println("Deserializing " + record); + System.err.println("Deserializing v2 " + record); + System.err.println(" key: " + BufferValue.bytesToHex(record.key())); + System.err.println(" value: " + BufferValue.bytesToHex(record.value())); + + final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); + final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); + final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); + + final int priorValueLength = valueAndTime.getInt(); + final byte[] priorValue; + if (priorValueLength == -1) { + priorValue = null; + } else { + priorValue = new byte[priorValueLength]; + valueAndTime.get(priorValue); + } + final long time = valueAndTime.getLong(); + final BufferValue bufferValue = new BufferValue(priorValue, change.oldValue, change.newValue, contextualRecord.recordContext()); + System.err.println(" got: " + bufferValue); + cleanPut(time, key, bufferValue); + } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_3_CHANGELOG_HEADER_VALUE)) { + // in this case, the changelog value is a serialized BufferValue + System.err.println("Deserializing v3 " + record); System.err.println(" key: " + BufferValue.bytesToHex(record.key())); System.err.println(" value: " + BufferValue.bytesToHex(record.value())); final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); + System.err.println(" got: " + bufferValue); final long time = valueAndTime.getLong(); cleanPut(time, key, bufferValue); } else { diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 3daf0dd01e947..a45f4c408add4 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -302,9 +302,9 @@ def test_simple_upgrade_downgrade(self, from_version, to_version): node = p.node with node.account.monitor_log(p.STDOUT_FILE) as monitor: p.stop() - monitor.wait_until("UPGRADE-TEST-CLIENT-CLOSED", + monitor.wait_until("SMOKE-TEST-CLIENT-CLOSED", timeout_sec=60, - err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) + err_msg="Never saw output 'SMOKE-TEST-CLIENT-CLOSED' on " + str(node.account)) def test_version_probing_upgrade(self): """ From e29728f1312b1fdbdf3294ee12439b7f5414f23b Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 23:09:08 -0500 Subject: [PATCH 07/23] wip --- .../kafka/streams/tests/SmokeTestClient.java | 9 --------- tests/kafkatest/services/streams.py | 16 +++++++++++----- .../tests/streams/streams_upgrade_test.py | 15 ++++++++------- 3 files changed, 19 insertions(+), 21 deletions(-) diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index 77458877bc51a..0cacf986d89cb 100644 --- a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.tests; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.KafkaThread; @@ -160,13 +158,6 @@ private Properties getStreamsConfig(final Properties props) { final Properties fullProps = new Properties(props); fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); - fullProps.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); - fullProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); - fullProps.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); - fullProps.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); - fullProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - fullProps.put(ProducerConfig.ACKS_CONFIG, "all"); fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); fullProps.putAll(props); return fullProps; diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index a913b1c65201e..354db13f8460a 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -305,7 +305,7 @@ def start_node(self, node): class StreamsSmokeTestBaseService(StreamsTestBaseService): """Base class for Streams Smoke Test services providing some common settings and functionality""" - def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', num_threads = 3): + def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', num_threads = 3, replication_factor = 3): super(StreamsSmokeTestBaseService, self).__init__(test_context, kafka, "org.apache.kafka.streams.tests.StreamsSmokeTest", @@ -314,6 +314,7 @@ def __init__(self, test_context, kafka, command, processing_guarantee = 'at_leas self.PROCESSING_GUARANTEE = processing_guarantee self.KAFKA_STREAMS_VERSION = "" self.UPGRADE_FROM = None + self.REPLICATION_FACTOR = replication_factor def set_version(self, kafka_streams_version): self.KAFKA_STREAMS_VERSION = kafka_streams_version @@ -321,12 +322,17 @@ def set_version(self, kafka_streams_version): def set_upgrade_from(self, upgrade_from): self.UPGRADE_FROM = upgrade_from - def prop_file(self): properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT, streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(), streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE, - streams_property.NUM_THREADS: self.NUM_THREADS} + streams_property.NUM_THREADS: self.NUM_THREADS, + "replication.factor": self.REPLICATION_FACTOR, + "num.standby.replicas": 2, + "buffered.records.per.partition": 100, + "commit.interval.ms": 1000, + "auto.offset.reset": "earliest", + "acks": "all"} if self.UPGRADE_FROM is not None: properties['upgrade.from'] = self.UPGRADE_FROM @@ -408,8 +414,8 @@ def start_cmd(self, node): return cmd class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): - def __init__(self, test_context, kafka, processing_guarantee, num_threads = 3): - super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, num_threads) + def __init__(self, test_context, kafka, processing_guarantee, num_threads = 3, replication_factor = 3): + super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, num_threads, replication_factor) class StreamsEosTestDriverService(StreamsEosTestBaseService): def __init__(self, test_context, kafka): diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index a45f4c408add4..6f66d165e5e0a 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -38,6 +38,8 @@ backward_compatible_metadata_2_versions = [str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)] metadata_3_or_higher_versions = [str(LATEST_2_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(DEV_VERSION)] +dev_version = [str(DEV_VERSION)] + """ After each release one should first check that the released version has been uploaded to https://s3-us-west-2.amazonaws.com/kafka-packages/ which is the url used by system test to download jars; @@ -242,9 +244,8 @@ def test_metadata_upgrade(self, from_version, to_version): timeout_sec=60, err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) - @matrix(from_version=metadata_2_versions, to_version=metadata_2_versions) - @matrix(from_version=metadata_3_or_higher_versions, to_version=metadata_3_or_higher_versions) - def test_simple_upgrade_downgrade(self, from_version, to_version): + @matrix(from_version=metadata_3_or_higher_versions, to_version=dev_version) + def test_app_upgrade(self, from_version, to_version): """ Starts 3 KafkaStreams instances with , and upgrades one-by-one to """ @@ -255,7 +256,7 @@ def test_simple_upgrade_downgrade(self, from_version, to_version): self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() - self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, topics={ + self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics={ 'echo' : { 'partitions': 5, 'replication-factor': 1 }, 'data' : { 'partitions': 5, 'replication-factor': 1 }, 'min' : { 'partitions': 5, 'replication-factor': 1 }, @@ -275,9 +276,9 @@ def test_simple_upgrade_downgrade(self, from_version, to_version): self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.driver.disable_auto_terminate() - self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once") - self.processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once") - self.processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once") + self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) + self.processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) + self.processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) self.driver.start() self.start_all_nodes_with(from_version) From 738eef739b53f395f4d7769f946fc8ba70ea89d7 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 23:21:34 -0500 Subject: [PATCH 08/23] cleanup --- .../streams/state/internals/BufferValue.java | 57 +- .../InMemoryTimeOrderedKeyValueBuffer.java | 15 - .../SmokeTestDriverIntegrationTest.java | 4 - .../kafka/streams/tests/SmokeTestClient.java | 120 ++-- .../kafka/streams/tests/SmokeTestDriver.java | 20 +- .../kafka/streams/tests/SmokeTestClient.java | 305 --------- .../kafka/streams/tests/SmokeTestDriver.java | 622 ------------------ .../kafka/streams/tests/SmokeTestUtil.java | 134 ---- .../kafka/streams/tests/StreamsSmokeTest.java | 99 --- 9 files changed, 87 insertions(+), 1289 deletions(-) delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java index 9ace9368dd2ab..225dbb5d803e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java @@ -16,16 +16,12 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.kstream.internals.FullChangeSerde; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Objects; -import static java.util.Objects.requireNonNull; - public final class BufferValue { private static final int NULL_VALUE_SENTINEL = -1; private static final int OLD_PREV_DUPLICATE_VALUE_SENTINEL = -2; @@ -34,53 +30,6 @@ public final class BufferValue { private final byte[] newValue; private final ProcessorRecordContext recordContext; - public static void main(String[] args) { - final String str="00000172D1434655000000000000006F0000000C6C6F67732D696E6772657373000000180000000000000059FFFFFFFF00000051080110AC051A2436346139616437662D313838352D346234342D613163302D633134346565633162636665222436346139616437662D313838352D346234342D613163302D633134346565633162636665FFFFFFFF00000172D1434655"; - final byte[] arr = hexStringToByteArray(str); - - // in this case, the changelog value is a serialized ContextualRecord - final ByteBuffer timeAndValue = ByteBuffer.wrap(arr); - final long time = timeAndValue.getLong(); - final byte[] changelogValue = new byte[arr.length - 8]; - timeAndValue.get(changelogValue); - - final ContextualRecord contextualRecord = ContextualRecord.deserialize(ByteBuffer.wrap(changelogValue)); - final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); - - System.out.println(change); - - final BufferValue deserialize = deserialize(ByteBuffer.wrap(arr)); - System.out.println(deserialize); - } - - public static byte[] hexStringToByteArray(String s) { - int len = s.length(); - byte[] data = new byte[len / 2]; - for (int i = 0; i < len; i += 2) { - data[i / 2] = (byte) ((Character.digit(s.charAt(i), 16) << 4) - + Character.digit(s.charAt(i+1), 16)); - } - return data; - } - - private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); - - public static String bytesToHex(byte[] bytes) { - if (bytes == null) { - return "null"; - } - final int length = bytes.length; - char[] hexChars = new char[2 + length * 2]; - hexChars[0] = '0'; - hexChars[1] = 'x'; - for (int j = 2; j < length; j++) { - int v = bytes[j] & 0xFF; - hexChars[j * 2] = HEX_ARRAY[v >>> 4]; - hexChars[j * 2 + 1] = HEX_ARRAY[v & 0x0F]; - } - return new String(hexChars); - } - BufferValue(final byte[] priorValue, final byte[] oldValue, final byte[] newValue, @@ -222,9 +171,9 @@ public int hashCode() { @Override public String toString() { return "BufferValue{" + - "priorValue=" + bytesToHex(priorValue) + - ", oldValue=" + bytesToHex(oldValue) + - ", newValue=" + bytesToHex(newValue) + + "priorValue=" + Arrays.toString(priorValue) + + ", oldValue=" + Arrays.toString(oldValue) + + ", newValue=" + Arrays.toString(newValue) + ", recordContext=" + recordContext + '}'; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index ff71da6543b64..7d3798946d72c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -261,10 +261,6 @@ private void logValue(final Bytes key, final BufferKey bufferKey, final BufferVa final ByteBuffer buffer = value.serialize(sizeOfBufferTime); buffer.putLong(bufferKey.time()); final byte[] array = buffer.array(); - System.err.println("Serializing v3:"); - System.err.println(" key: " + BufferValue.bytesToHex(key.get())); - System.err.println(" value: " + value); - System.err.println(" serialized: " + BufferValue.bytesToHex(array)); ((RecordCollector.Supplier) context).recordCollector().send( changelogTopic, key, @@ -367,11 +363,6 @@ private void restoreBatch(final Collection> batch ) ); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_2_CHANGELOG_HEADER_VALUE)) { - // in this case, the changelog value is a serialized BufferValue - System.err.println("Deserializing v2 " + record); - System.err.println(" key: " + BufferValue.bytesToHex(record.key())); - System.err.println(" value: " + BufferValue.bytesToHex(record.value())); - final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); @@ -386,17 +377,11 @@ private void restoreBatch(final Collection> batch } final long time = valueAndTime.getLong(); final BufferValue bufferValue = new BufferValue(priorValue, change.oldValue, change.newValue, contextualRecord.recordContext()); - System.err.println(" got: " + bufferValue); cleanPut(time, key, bufferValue); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_3_CHANGELOG_HEADER_VALUE)) { - // in this case, the changelog value is a serialized BufferValue - System.err.println("Deserializing v3 " + record); - System.err.println(" key: " + BufferValue.bytesToHex(record.key())); - System.err.println(" value: " + BufferValue.bytesToHex(record.value())); final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); - System.err.println(" got: " + bufferValue); final long time = valueAndTime.getLong(); cleanPut(time, key, bufferValue); } else { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java index c2bd7a5d24615..0e4e78ff68f7f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java @@ -104,10 +104,6 @@ public void shouldWorkWithRebalance() throws InterruptedException { clients.add(smokeTestClient); smokeTestClient.start(props); - while (!clients.get(clients.size() - 1).started()) { - Thread.sleep(100); - } - // let the oldest client die of "natural causes" if (clients.size() >= 3) { final SmokeTestClient client = clients.remove(0); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index db243fdd849c9..c4bd439bdb346 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -21,6 +21,8 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -40,9 +42,14 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.test.TestUtils; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; import java.time.Duration; import java.time.Instant; import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; @@ -50,14 +57,40 @@ public class SmokeTestClient extends SmokeTestUtil { private final String name; - private Thread thread; private KafkaStreams streams; private boolean uncaughtException = false; private boolean started; - private boolean closed; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } public SmokeTestClient(final String name) { - super(); this.name = name; } @@ -70,17 +103,43 @@ public boolean closed() { } public void start(final Properties streamsProperties) { - streams = createKafkaStreams(streamsProperties); + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + streams.setUncaughtExceptionHandler((t, e) -> { System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + e.printStackTrace(System.out); uncaughtException = true; - e.printStackTrace(); + streams.close(Duration.ofSeconds(30)); }); - Exit.addShutdownHook("streams-shutdown-hook", () -> close()); + addShutdownHook("streams-shutdown-hook", this::close); - thread = new Thread(() -> streams.start()); - thread.start(); + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); } public void closeAsync() { @@ -88,17 +147,14 @@ public void closeAsync() { } public void close() { - streams.close(Duration.ofSeconds(5)); - // do not remove these printouts since they are needed for health scripts - if (!uncaughtException) { + final boolean wasClosed = streams.close(Duration.ofMinutes(1)); + + if (wasClosed && !uncaughtException) { System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } - try { - thread.join(); - } catch (final Exception ex) { - // do not remove these printouts since they are needed for health scripts + } else if (wasClosed) { System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - // ignore + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); } } @@ -106,39 +162,11 @@ private Properties getStreamsConfig(final Properties props) { final Properties fullProps = new Properties(props); fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); - fullProps.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); - fullProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); - fullProps.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); - fullProps.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); - fullProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - fullProps.put(ProducerConfig.ACKS_CONFIG, "all"); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); fullProps.putAll(props); return fullProps; } - private KafkaStreams createKafkaStreams(final Properties props) { - final Topology build = getTopology(); - final KafkaStreams streamsClient = new KafkaStreams(build, getStreamsConfig(props)); - streamsClient.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - streamsClient.setUncaughtExceptionHandler((t, e) -> { - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - streamsClient.close(Duration.ofSeconds(30)); - }); - - return streamsClient; - } - public Topology getTopology() { final StreamsBuilder builder = new StreamsBuilder(); final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java index ac83cd95ebaaf..6fd70e06c0dc0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -487,14 +487,14 @@ private static boolean verify(final PrintStream resultStream, if (printResults) { resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); @@ -525,8 +525,8 @@ private static boolean verifySuppressed(final PrintStream resultStream, if (printResults) resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); return false; } diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java deleted file mode 100644 index 77458877bc51a..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ /dev/null @@ -1,305 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.Topology; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Grouped; -import org.apache.kafka.streams.kstream.KGroupedStream; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; -import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.streams.state.WindowStore; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; - -public class SmokeTestClient extends SmokeTestUtil { - - private final String name; - - private KafkaStreams streams; - private boolean uncaughtException = false; - private boolean started; - private volatile boolean closed; - - private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); } else { - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } - } - - private static File tempDirectory() { - final String prefix = "kafka-"; - final File file; - try { - file = Files.createTempDirectory(prefix).toFile(); - } catch (final IOException ex) { - throw new RuntimeException("Failed to create a temp dir", ex); - } - file.deleteOnExit(); - - addShutdownHook("delete-temp-file-shutdown-hook", () -> { - try { - Utils.delete(file); - } catch (final IOException e) { - System.out.println("Error deleting " + file.getAbsolutePath()); - e.printStackTrace(System.out); - } - }); - - return file; - } - - public SmokeTestClient(final String name) { - this.name = name; - } - - public boolean started() { - return started; - } - - public boolean closed() { - return closed; - } - - public void start(final Properties streamsProperties) { - final Topology build = getTopology(); - streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); - - final CountDownLatch countDownLatch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; - countDownLatch.countDown(); - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - - streams.setUncaughtExceptionHandler((t, e) -> { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - e.printStackTrace(System.out); - uncaughtException = true; - streams.close(Duration.ofSeconds(30)); - }); - - addShutdownHook("streams-shutdown-hook", this::close); - - streams.start(); - try { - if (!countDownLatch.await(1, TimeUnit.MINUTES)) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); - } - } catch (final InterruptedException e) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); - e.printStackTrace(System.out); - } - System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); - System.out.println(name + " started at " + Instant.now()); - } - - public void closeAsync() { - streams.close(Duration.ZERO); - } - - public void close() { - final boolean closed = streams.close(Duration.ofMinutes(1)); - - if (closed && !uncaughtException) { - System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } else if (closed) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - } else { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); - } - } - - private Properties getStreamsConfig(final Properties props) { - final Properties fullProps = new Properties(props); - fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); - fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); - fullProps.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); - fullProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); - fullProps.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); - fullProps.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); - fullProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - fullProps.put(ProducerConfig.ACKS_CONFIG, "all"); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); - fullProps.putAll(props); - return fullProps; - } - - public Topology getTopology() { - final StreamsBuilder builder = new StreamsBuilder(); - final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); - final KStream source = builder.stream("data", stringIntConsumed); - source.filterNot((k, v) -> k.equals("flush")) - .to("echo", Produced.with(stringSerde, intSerde)); - final KStream data = source.filter((key, value) -> value == null || value != END); - data.process(SmokeTestUtil.printProcessorSupplier("data", name)); - - // min - final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); - - final KTable, Integer> minAggregation = groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) - .aggregate( - () -> Integer.MAX_VALUE, - (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, - Materialized - .>as("uwin-min") - .withValueSerde(intSerde) - .withRetention(Duration.ofHours(25)) - ); - - streamify(minAggregation, "min-raw"); - - streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); - - minAggregation - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("min", Produced.with(stringSerde, intSerde)); - - final KTable, Integer> smallWindowSum = groupedData - .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) - .reduce((l, r) -> l + r); - - streamify(smallWindowSum, "sws-raw"); - streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); - - final KTable minTable = builder.table( - "min", - Consumed.with(stringSerde, intSerde), - Materialized.as("minStoreName")); - - minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); - - // max - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> Integer.MIN_VALUE, - (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, - Materialized.>as("uwin-max").withValueSerde(intSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("max", Produced.with(stringSerde, intSerde)); - - final KTable maxTable = builder.table( - "max", - Consumed.with(stringSerde, intSerde), - Materialized.as("maxStoreName")); - maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); - - // sum - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> 0L, - (aggKey, value, aggregate) -> (long) value + aggregate, - Materialized.>as("win-sum").withValueSerde(longSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("sum", Produced.with(stringSerde, longSerde)); - - final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); - final KTable sumTable = builder.table("sum", stringLongConsumed); - sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); - - // cnt - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .count(Materialized.as("uwin-cnt")) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("cnt", Produced.with(stringSerde, longSerde)); - - final KTable cntTable = builder.table( - "cnt", - Consumed.with(stringSerde, longSerde), - Materialized.as("cntStoreName")); - cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); - - // dif - maxTable - .join( - minTable, - (value1, value2) -> value1 - value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("dif", Produced.with(stringSerde, intSerde)); - - // avg - sumTable - .join( - cntTable, - (value1, value2) -> (double) value1 / (double) value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("avg", Produced.with(stringSerde, doubleSerde)); - - // test repartition - final Agg agg = new Agg(); - cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) - .aggregate(agg.init(), agg.adder(), agg.remover(), - Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) - .withKeySerde(Serdes.String()) - .withValueSerde(Serdes.Long())) - .toStream() - .to("tagg", Produced.with(stringSerde, longSerde)); - - return builder.build(); - } - - private static void streamify(final KTable, Integer> windowedTable, final String topic) { - windowedTable - .toStream() - .filterNot((k, v) -> k.key().equals("flush")) - .map((key, value) -> new KeyValue<>(key.toString(), value)) - .to(topic, Produced.with(stringSerde, intSerde)); - } -} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java deleted file mode 100644 index ac83cd95ebaaf..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ /dev/null @@ -1,622 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.utils.Utils; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; - -public class SmokeTestDriver extends SmokeTestUtil { - private static final String[] TOPICS = { - "data", - "echo", - "max", - "min", "min-suppressed", "min-raw", - "dif", - "sum", - "sws-raw", "sws-suppressed", - "cnt", - "avg", - "tagg" - }; - - private static final int MAX_RECORD_EMPTY_RETRIES = 30; - - private static class ValueList { - public final String key; - private final int[] values; - private int index; - - ValueList(final int min, final int max) { - key = min + "-" + max; - - values = new int[max - min + 1]; - for (int i = 0; i < values.length; i++) { - values[i] = min + i; - } - // We want to randomize the order of data to test not completely predictable processing order - // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) - // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window - shuffle(values, 10); - - index = 0; - } - - int next() { - return (index < values.length) ? values[index++] : -1; - } - } - - public static String[] topics() { - return Arrays.copyOf(TOPICS, TOPICS.length); - } - - static void generatePerpetually(final String kafka, - final int numKeys, - final int maxRecordsPerKey) { - final Properties producerProps = generatorProperties(kafka); - - int numRecordsProduced = 0; - - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - } - - final Random rand = new Random(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (true) { - final int index = rand.nextInt(numKeys); - final String key = data[index].key; - final int value = data[index].next(); - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record); - - numRecordsProduced++; - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(2); - } - } - } - - public static Map> generate(final String kafka, - final int numKeys, - final int maxRecordsPerKey, - final Duration timeToSpend) { - final Properties producerProps = generatorProperties(kafka); - - - int numRecordsProduced = 0; - - final Map> allData = new HashMap<>(); - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - allData.put(data[i].key, new HashSet<>()); - } - final Random rand = new Random(); - - int remaining = data.length; - - final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; - - List> needRetry = new ArrayList<>(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (remaining > 0) { - final int index = rand.nextInt(remaining); - final String key = data[index].key; - final int value = data[index].next(); - - if (value < 0) { - remaining--; - data[index] = data[remaining]; - } else { - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record, new TestCallback(record, needRetry)); - - numRecordsProduced++; - allData.get(key).add(value); - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(Math.max(recordPauseTime, 2)); - } - } - producer.flush(); - - int remainingRetries = 5; - while (!needRetry.isEmpty()) { - final List> needRetry2 = new ArrayList<>(); - for (final ProducerRecord record : needRetry) { - System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); - producer.send(record, new TestCallback(record, needRetry2)); - } - producer.flush(); - needRetry = needRetry2; - - if (--remainingRetries == 0 && !needRetry.isEmpty()) { - System.err.println("Failed to produce all records after multiple retries"); - Exit.exit(1); - } - } - - // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out - // all suppressed records. - final List partitions = producer.partitionsFor("data"); - for (final PartitionInfo partition : partitions) { - producer.send(new ProducerRecord<>( - partition.topic(), - partition.partition(), - System.currentTimeMillis() + Duration.ofDays(2).toMillis(), - stringSerde.serializer().serialize("", "flush"), - intSerde.serializer().serialize("", 0) - )); - } - } - return Collections.unmodifiableMap(allData); - } - - private static Properties generatorProperties(final String kafka) { - final Properties producerProps = new Properties(); - producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); - return producerProps; - } - - private static class TestCallback implements Callback { - private final ProducerRecord originalRecord; - private final List> needRetry; - - TestCallback(final ProducerRecord originalRecord, - final List> needRetry) { - this.originalRecord = originalRecord; - this.needRetry = needRetry; - } - - @Override - public void onCompletion(final RecordMetadata metadata, final Exception exception) { - if (exception != null) { - if (exception instanceof TimeoutException) { - needRetry.add(originalRecord); - } else { - exception.printStackTrace(); - Exit.exit(1); - } - } - } - } - - private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { - final Random rand = new Random(); - for (int i = 0; i < data.length; i++) { - // we shuffle data within windowSize - final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; - - // swap - final int tmp = data[i]; - data[i] = data[j]; - data[j] = tmp; - } - } - - public static class NumberDeserializer implements Deserializer { - @Override - public Number deserialize(final String topic, final byte[] data) { - final Number value; - switch (topic) { - case "data": - case "echo": - case "min": - case "min-raw": - case "min-suppressed": - case "sws-raw": - case "sws-suppressed": - case "max": - case "dif": - value = intSerde.deserializer().deserialize(topic, data); - break; - case "sum": - case "cnt": - case "tagg": - value = longSerde.deserializer().deserialize(topic, data); - break; - case "avg": - value = doubleSerde.deserializer().deserialize(topic, data); - break; - default: - throw new RuntimeException("unknown topic: " + topic); - } - return value; - } - } - - public static VerificationResult verify(final String kafka, - final Map> inputs, - final int maxRecordsPerKey) { - final Properties props = new Properties(); - props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); - props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); - - final KafkaConsumer consumer = new KafkaConsumer<>(props); - final List partitions = getAllPartitions(consumer, TOPICS); - consumer.assign(partitions); - consumer.seekToBeginning(partitions); - - final int recordsGenerated = inputs.size() * maxRecordsPerKey; - int recordsProcessed = 0; - final Map processed = - Stream.of(TOPICS) - .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); - - final Map>>> events = new HashMap<>(); - - VerificationResult verificationResult = new VerificationResult(false, "no results yet"); - int retry = 0; - final long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { - final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (records.isEmpty() && recordsProcessed >= recordsGenerated) { - verificationResult = verifyAll(inputs, events, false); - if (verificationResult.passed()) { - break; - } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { - System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); - break; - } else { - System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); - } - } else { - System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); - - retry = 0; - for (final ConsumerRecord record : records) { - final String key = record.key(); - - final String topic = record.topic(); - processed.get(topic).incrementAndGet(); - - if (topic.equals("echo")) { - recordsProcessed++; - if (recordsProcessed % 100 == 0) { - System.out.println("Echo records processed = " + recordsProcessed); - } - } - - events.computeIfAbsent(topic, t -> new HashMap<>()) - .computeIfAbsent(key, k -> new LinkedList<>()) - .add(record); - } - - System.out.println(processed); - } - } - consumer.close(); - final long finished = System.currentTimeMillis() - start; - System.out.println("Verification time=" + finished); - System.out.println("-------------------"); - System.out.println("Result Verification"); - System.out.println("-------------------"); - System.out.println("recordGenerated=" + recordsGenerated); - System.out.println("recordProcessed=" + recordsProcessed); - - if (recordsProcessed > recordsGenerated) { - System.out.println("PROCESSED-MORE-THAN-GENERATED"); - } else if (recordsProcessed < recordsGenerated) { - System.out.println("PROCESSED-LESS-THAN-GENERATED"); - } - - boolean success; - - final Map> received = - events.get("echo") - .entrySet() - .stream() - .map(entry -> mkEntry( - entry.getKey(), - entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) - ) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - success = inputs.equals(received); - - if (success) { - System.out.println("ALL-RECORDS-DELIVERED"); - } else { - int missedCount = 0; - for (final Map.Entry> entry : inputs.entrySet()) { - missedCount += received.get(entry.getKey()).size(); - } - System.out.println("missedRecords=" + missedCount); - } - - // give it one more try if it's not already passing. - if (!verificationResult.passed()) { - verificationResult = verifyAll(inputs, events, true); - } - success &= verificationResult.passed(); - - System.out.println(verificationResult.result()); - - System.out.println(success ? "SUCCESS" : "FAILURE"); - return verificationResult; - } - - public static class VerificationResult { - private final boolean passed; - private final String result; - - VerificationResult(final boolean passed, final String result) { - this.passed = passed; - this.result = result; - } - - public boolean passed() { - return passed; - } - - public String result() { - return result; - } - } - - private static VerificationResult verifyAll(final Map> inputs, - final Map>>> events, - final boolean printResults) { - final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - boolean pass; - try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { - pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); - pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); - pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { - final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); - return getMin(unwindowedKey); - }, printResults); - pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); - pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); - pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); - pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); - pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); - pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); - pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); - } - return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); - } - - private static boolean verify(final PrintStream resultStream, - final String topic, - final Map> inputData, - final Map>>> events, - final Function keyToExpectation, - final boolean printResults) { - final Map>> observedInputEvents = events.get("data"); - final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); - if (outputEvents.isEmpty()) { - resultStream.println(topic + " is empty"); - return false; - } else { - resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); - - if (outputEvents.size() != inputData.size()) { - resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", - outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); - return false; - } - for (final Map.Entry>> entry : outputEvents.entrySet()) { - final String key = entry.getKey(); - final Number expected = keyToExpectation.apply(key); - final Number actual = entry.getValue().getLast().value(); - if (!expected.equals(actual)) { - resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); - - if (printResults) { - resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); - - if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) - resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); - } - - return false; - } - } - return true; - } - } - - - private static boolean verifySuppressed(final PrintStream resultStream, - @SuppressWarnings("SameParameterValue") final String topic, - final Map>>> events, - final boolean printResults) { - resultStream.println("verifying suppressed " + topic); - final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); - for (final Map.Entry>> entry : topicEvents.entrySet()) { - if (entry.getValue().size() != 1) { - final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); - final String key = entry.getKey(); - final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); - resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", - key, - indent("\t\t", entry.getValue())); - - if (printResults) - resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); - - return false; - } - } - return true; - } - - private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, - final Iterable> list) { - final StringBuilder stringBuilder = new StringBuilder(); - for (final ConsumerRecord record : list) { - stringBuilder.append(prefix).append(record).append('\n'); - } - return stringBuilder.toString(); - } - - private static Long getSum(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) * (max - min + 1L) / 2L; - } - - private static Double getAvg(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) / 2.0; - } - - - private static boolean verifyTAgg(final PrintStream resultStream, - final Map> allData, - final Map>> taggEvents, - final boolean printResults) { - if (taggEvents == null) { - resultStream.println("tagg is missing"); - return false; - } else if (taggEvents.isEmpty()) { - resultStream.println("tagg is empty"); - return false; - } else { - resultStream.println("verifying tagg"); - - // generate expected answer - final Map expected = new HashMap<>(); - for (final String key : allData.keySet()) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - final String cnt = Long.toString(max - min + 1L); - - expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); - } - - // check the result - for (final Map.Entry>> entry : taggEvents.entrySet()) { - final String key = entry.getKey(); - Long expectedCount = expected.remove(key); - if (expectedCount == null) { - expectedCount = 0L; - } - - if (entry.getValue().getLast().value().longValue() != expectedCount) { - resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); - - if (printResults) - resultStream.println("\t taggEvents: " + entry.getValue()); - return false; - } - } - - } - return true; - } - - private static Number getMin(final String key) { - return Integer.parseInt(key.split("-")[0]); - } - - private static Number getMax(final String key) { - return Integer.parseInt(key.split("-")[1]); - } - - private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { - final List partitions = new ArrayList<>(); - - for (final String topic : topics) { - for (final PartitionInfo info : consumer.partitionsFor(topic)) { - partitions.add(new TopicPartition(info.topic(), info.partition())); - } - } - return partitions; - } - -} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java deleted file mode 100644 index e8ec04c23ca52..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -import java.time.Instant; - -public class SmokeTestUtil { - - final static int END = Integer.MAX_VALUE; - - static ProcessorSupplier printProcessorSupplier(final String topic) { - return printProcessorSupplier(topic, ""); - } - - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return new ProcessorSupplier() { - @Override - public Processor get() { - return new AbstractProcessor() { - private int numRecordsProcessed = 0; - private long smallestOffset = Long.MAX_VALUE; - private long largestOffset = Long.MIN_VALUE; - - @Override - public void init(final ProcessorContext context) { - super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); - System.out.flush(); - numRecordsProcessed = 0; - smallestOffset = Long.MAX_VALUE; - largestOffset = Long.MIN_VALUE; - } - - @Override - public void process(final Object key, final Object value) { - numRecordsProcessed++; - if (numRecordsProcessed % 100 == 0) { - System.out.printf("%s: %s%n", name, Instant.now()); - System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); - } - - if (smallestOffset > context().offset()) { - smallestOffset = context().offset(); - } - if (largestOffset < context().offset()) { - largestOffset = context().offset(); - } - } - - @Override - public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); - System.out.println("processed " + numRecordsProcessed + " records"); - final long processed; - if (largestOffset >= smallestOffset) { - processed = 1L + largestOffset - smallestOffset; - } else { - processed = 0L; - } - System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); - System.out.flush(); - } - }; - } - }; - } - - public static final class Unwindow implements KeyValueMapper, V, K> { - @Override - public K apply(final Windowed winKey, final V value) { - return winKey.key(); - } - } - - public static class Agg { - - KeyValueMapper> selector() { - return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); - } - - public Initializer init() { - return () -> 0L; - } - - Aggregator adder() { - return (aggKey, value, aggregate) -> aggregate + value; - } - - Aggregator remover() { - return (aggKey, value, aggregate) -> aggregate - value; - } - } - - public static Serde stringSerde = Serdes.String(); - - public static Serde intSerde = Serdes.Integer(); - - static Serde longSerde = Serdes.Long(); - - static Serde doubleSerde = Serdes.Double(); - - public static void sleep(final long duration) { - try { - Thread.sleep(duration); - } catch (final Exception ignore) { } - } - -} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java deleted file mode 100644 index 07c7d5d82ebff..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.StreamsConfig; - -import java.io.IOException; -import java.time.Duration; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; - -import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; -import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; - -public class StreamsSmokeTest { - - /** - * args ::= kafka propFileName command disableAutoTerminate - * command := "run" | "process" - * - * @param args - */ - public static void main(final String[] args) throws IOException { - if (args.length < 2) { - System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); - System.exit(1); - } - - final String propFileName = args[0]; - final String command = args[1]; - final boolean disableAutoTerminate = args.length > 2; - - final Properties streamsProperties = Utils.loadProps(propFileName); - final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); - - if (kafka == null) { - System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - System.exit(1); - } - - if ("process".equals(command)) { - if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { - - System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + - StreamsConfig.EXACTLY_ONCE); - - System.exit(1); - } - } - - System.out.println("StreamsTest instance started (StreamsSmokeTest)"); - System.out.println("command=" + command); - System.out.println("props=" + streamsProperties); - System.out.println("disableAutoTerminate=" + disableAutoTerminate); - - switch (command) { - case "run": - // this starts the driver (data generation and result verification) - final int numKeys = 10; - final int maxRecordsPerKey = 500; - if (disableAutoTerminate) { - generatePerpetually(kafka, numKeys, maxRecordsPerKey); - } else { - // slow down data production to span 30 seconds so that system tests have time to - // do their bounces, etc. - final Map> allData = - generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); - SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); - } - break; - case "process": - // this starts the stream processing app - new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); - break; - default: - System.out.println("unknown command: " + command); - } - } - -} From 9cd927abc299fafa599324675900171e1f8b6360 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Wed, 24 Jun 2020 23:27:17 -0500 Subject: [PATCH 09/23] cleanup --- .../org/apache/kafka/streams/tests/SmokeTestClient.java | 8 +++----- .../org/apache/kafka/streams/tests/SmokeTestClient.java | 4 +++- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index c4bd439bdb346..c518491f5b54f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -16,11 +16,8 @@ */ package org.apache.kafka.streams.tests; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KafkaStreams; @@ -40,7 +37,6 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.WindowStore; -import org.apache.kafka.test.TestUtils; import java.io.File; import java.io.IOException; @@ -63,7 +59,9 @@ public class SmokeTestClient extends SmokeTestUtil { private volatile boolean closed; private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); } else { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { Runtime.getRuntime().addShutdownHook(new Thread(runnable)); } } diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index 0cacf986d89cb..ced1369af3855 100644 --- a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -59,7 +59,9 @@ public class SmokeTestClient extends SmokeTestUtil { private volatile boolean closed; private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); } else { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { Runtime.getRuntime().addShutdownHook(new Thread(runnable)); } } From 23964583a267948249ae300466f15c2de9b1b647 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 12:28:57 -0500 Subject: [PATCH 10/23] fix test --- .../state/internals/TimeOrderedKeyValueBufferTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 25a44c486f43b..a9664da72f6c3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -62,8 +62,8 @@ @RunWith(Parameterized.class) public class TimeOrderedKeyValueBufferTest> { - private static final RecordHeaders V_2_CHANGELOG_HEADERS = - new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 2})}); + private static final RecordHeaders CHANGELOG_HEADERS = + new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 3})}); private static final String APP_ID = "test-app"; private final Function bufferSupplier; @@ -347,14 +347,14 @@ public void shouldFlush() { null, "zxcv", new KeyValue<>(1L, getBufferValue("3gon4i", 1)), - V_2_CHANGELOG_HEADERS + CHANGELOG_HEADERS ), new ProducerRecord<>(APP_ID + "-" + testName + "-changelog", 0, null, "asdf", new KeyValue<>(2L, getBufferValue("2093j", 0)), - V_2_CHANGELOG_HEADERS + CHANGELOG_HEADERS ) ))); From 29ab2070b0a7e8124de91a8cc2ce01b64ac6c1eb Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 14:29:29 -0500 Subject: [PATCH 11/23] direct encoding of restore test vx --- .../TimeOrderedKeyValueBufferTest.java | 48 +++++++++++++++---- 1 file changed, 38 insertions(+), 10 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index a9664da72f6c3..9754fcf3e3e55 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -361,8 +361,30 @@ public void shouldFlush() { cleanup(context, buffer); } + private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); + + private static String bytesToHex(byte[] bytes) { + char[] hexChars = new char[bytes.length * 2]; + for (int j = 0; j < bytes.length; j++) { + int v = bytes[j] & 0xFF; + hexChars[j * 2] = HEX_ARRAY[v >>> 4]; + hexChars[j * 2 + 1] = HEX_ARRAY[v & 0x0F]; + } + return new String(hexChars); + } + + public static byte[] hexStringToByteArray(String s) { + int len = s.length(); + byte[] data = new byte[len / 2]; + for (int i = 0; i < len; i += 2) { + data[i / 2] = (byte) ((Character.digit(s.charAt(i), 16) << 4) + + Character.digit(s.charAt(i+1), 16)); + } + return data; + } + @Test - public void shouldRestoreOldFormat() { + public void shouldRestoreOldUnversionedFormat() { final TimeOrderedKeyValueBuffer buffer = bufferSupplier.apply(testName); final MockInternalProcessorContext context = makeContext(); buffer.init(context, buffer); @@ -372,12 +394,18 @@ public void shouldRestoreOldFormat() { context.setRecordContext(new ProcessorRecordContext(0, 0, 0, "", null)); - final FullChangeSerde serializer = FullChangeSerde.wrap(Serdes.String()); + final String toDeleteBinaryValue = "0000000000000000FFFFFFFF00000006646F6F6D6564"; + final byte[] toDeleteRecordValue = hexStringToByteArray(toDeleteBinaryValue); + + final String asdfBinaryValue = "0000000000000002FFFFFFFF0000000471776572"; + final byte[] asdfRecordValue = hexStringToByteArray(asdfBinaryValue); + + final String zxcvBinaryValue1 = "00000000000000010000000870726576696F757300000005656F34696D"; + final byte[] zxcvRecordValue1 = hexStringToByteArray(zxcvBinaryValue1); + + final String zxcvBinaryValue2 = "000000000000000100000005656F34696D000000046E657874"; + final byte[] zxcvRecordValue2 = hexStringToByteArray(zxcvBinaryValue2); - final byte[] todeleteValue = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(serializer.serializeParts(null, new Change<>("doomed", null))); - final byte[] asdfValue = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(serializer.serializeParts(null, new Change<>("qwer", null))); - final byte[] zxcvValue1 = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(serializer.serializeParts(null, new Change<>("eo4im", "previous"))); - final byte[] zxcvValue2 = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(serializer.serializeParts(null, new Change<>("next", "eo4im"))); stateRestoreCallback.restoreBatch(asList( new ConsumerRecord<>("changelog-topic", 0, @@ -388,7 +416,7 @@ public void shouldRestoreOldFormat() { -1, -1, "todelete".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + todeleteValue.length).putLong(0L).put(todeleteValue).array()), + toDeleteRecordValue), new ConsumerRecord<>("changelog-topic", 0, 1, @@ -398,7 +426,7 @@ public void shouldRestoreOldFormat() { -1, -1, "asdf".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + asdfValue.length).putLong(2L).put(asdfValue).array()), + asdfRecordValue), new ConsumerRecord<>("changelog-topic", 0, 2, @@ -408,7 +436,7 @@ public void shouldRestoreOldFormat() { -1, -1, "zxcv".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + zxcvValue1.length).putLong(1L).put(zxcvValue1).array()), + zxcvRecordValue1), new ConsumerRecord<>("changelog-topic", 0, 3, @@ -418,7 +446,7 @@ public void shouldRestoreOldFormat() { -1, -1, "zxcv".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + zxcvValue2.length).putLong(1L).put(zxcvValue2).array()) + zxcvRecordValue2) )); assertThat(buffer.numRecords(), is(3)); From dab0fc1c157fb6183469056dbf715007f6721880 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 14:49:56 -0500 Subject: [PATCH 12/23] direct encoding of restore test v1 --- .../TimeOrderedKeyValueBufferTest.java | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 9754fcf3e3e55..cf01c53a09587 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -515,16 +515,33 @@ public void shouldRestoreV1Format() { final RecordHeaders v1FlagHeaders = new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 1})}); final byte[] todeleteValue = getContextualRecord("doomed", 0).serialize(0).array(); + final byte[] asdfValue = getContextualRecord("qwer", 1).serialize(0).array(); + final FullChangeSerde fullChangeSerde = FullChangeSerde.wrap(Serdes.String()); + final byte[] zxcvValue1 = new ContextualRecord( FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(fullChangeSerde.serializeParts(null, new Change<>("3o4im", "previous"))), getContext(2L) ).serialize(0).array(); + final byte[] zxcvValue2 = new ContextualRecord( FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(fullChangeSerde.serializeParts(null, new Change<>("next", "3o4im"))), getContext(3L) ).serialize(0).array(); + + final String toDeleteBinary = "00000000000000000000000000000000000000000000000000000005746F70696300000000FFFFFFFF0000000EFFFFFFFF00000006646F6F6D6564"; + final byte[] toDeleteArray = hexStringToByteArray(toDeleteBinary); + + final String asdfBinary = "00000000000000020000000000000001000000000000000000000005746F70696300000000FFFFFFFF0000000CFFFFFFFF0000000471776572"; + final byte[] asdfArray = hexStringToByteArray(asdfBinary); + + final String zxcvBinary1 = "00000000000000010000000000000002000000000000000000000005746F70696300000000FFFFFFFF000000150000000870726576696F757300000005336F34696D"; + final byte[] zxcvArray1 = hexStringToByteArray(zxcvBinary1); + + final String zxcvBinary2 = "00000000000000010000000000000003000000000000000000000005746F70696300000000FFFFFFFF0000001100000005336F34696D000000046E657874"; + final byte[] zxcvArray2 = hexStringToByteArray(zxcvBinary2); + stateRestoreCallback.restoreBatch(asList( new ConsumerRecord<>("changelog-topic", 0, @@ -535,7 +552,7 @@ public void shouldRestoreV1Format() { -1, -1, "todelete".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + todeleteValue.length).putLong(0L).put(todeleteValue).array(), + toDeleteArray, v1FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -546,7 +563,7 @@ public void shouldRestoreV1Format() { -1, -1, "asdf".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + asdfValue.length).putLong(2L).put(asdfValue).array(), + asdfArray, v1FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -557,7 +574,7 @@ public void shouldRestoreV1Format() { -1, -1, "zxcv".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + zxcvValue1.length).putLong(1L).put(zxcvValue1).array(), + zxcvArray1, v1FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -568,7 +585,7 @@ public void shouldRestoreV1Format() { -1, -1, "zxcv".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + zxcvValue2.length).putLong(1L).put(zxcvValue2).array(), + zxcvArray2, v1FlagHeaders) )); From 776efb673d0b8b71f63d5bc92fc538d4bf9eeea2 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 15:19:11 -0500 Subject: [PATCH 13/23] add v2 and v3 --- .../TimeOrderedKeyValueBufferTest.java | 232 ++++++++++++------ 1 file changed, 156 insertions(+), 76 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index cf01c53a09587..9b8e50680d8d2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.kstream.internals.FullChangeSerde; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback; @@ -73,7 +72,7 @@ public static final class NullRejectingStringSerializer extends StringSerializer @Override public byte[] serialize(final String topic, final String data) { if (data == null) { - throw new IllegalArgumentException(); + throw new IllegalArgumentException("null data not allowed"); } return super.serialize(topic, data); } @@ -363,22 +362,22 @@ public void shouldFlush() { private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); - private static String bytesToHex(byte[] bytes) { - char[] hexChars = new char[bytes.length * 2]; + private static String bytesToHex(final byte[] bytes) { + final char[] hexChars = new char[bytes.length * 2]; for (int j = 0; j < bytes.length; j++) { - int v = bytes[j] & 0xFF; + final int v = bytes[j] & 0xFF; hexChars[j * 2] = HEX_ARRAY[v >>> 4]; hexChars[j * 2 + 1] = HEX_ARRAY[v & 0x0F]; } return new String(hexChars); } - public static byte[] hexStringToByteArray(String s) { - int len = s.length(); - byte[] data = new byte[len / 2]; + private static byte[] hexStringToByteArray(final String hexString) { + final int len = hexString.length(); + final byte[] data = new byte[len / 2]; for (int i = 0; i < len; i += 2) { - data[i / 2] = (byte) ((Character.digit(s.charAt(i), 16) << 4) - + Character.digit(s.charAt(i+1), 16)); + data[i / 2] = (byte) ((Character.digit(hexString.charAt(i), 16) << 4) + + Character.digit(hexString.charAt(i+1), 16)); } return data; } @@ -394,17 +393,13 @@ public void shouldRestoreOldUnversionedFormat() { context.setRecordContext(new ProcessorRecordContext(0, 0, 0, "", null)); + // These serialized formats were captured by running version 2.1 code. + // They verify that an upgrade from 2.1 will work. + // Do not change them. final String toDeleteBinaryValue = "0000000000000000FFFFFFFF00000006646F6F6D6564"; - final byte[] toDeleteRecordValue = hexStringToByteArray(toDeleteBinaryValue); - final String asdfBinaryValue = "0000000000000002FFFFFFFF0000000471776572"; - final byte[] asdfRecordValue = hexStringToByteArray(asdfBinaryValue); - final String zxcvBinaryValue1 = "00000000000000010000000870726576696F757300000005656F34696D"; - final byte[] zxcvRecordValue1 = hexStringToByteArray(zxcvBinaryValue1); - final String zxcvBinaryValue2 = "000000000000000100000005656F34696D000000046E657874"; - final byte[] zxcvRecordValue2 = hexStringToByteArray(zxcvBinaryValue2); stateRestoreCallback.restoreBatch(asList( new ConsumerRecord<>("changelog-topic", @@ -416,7 +411,7 @@ public void shouldRestoreOldUnversionedFormat() { -1, -1, "todelete".getBytes(UTF_8), - toDeleteRecordValue), + hexStringToByteArray(toDeleteBinaryValue)), new ConsumerRecord<>("changelog-topic", 0, 1, @@ -426,7 +421,7 @@ public void shouldRestoreOldUnversionedFormat() { -1, -1, "asdf".getBytes(UTF_8), - asdfRecordValue), + hexStringToByteArray(asdfBinaryValue)), new ConsumerRecord<>("changelog-topic", 0, 2, @@ -436,7 +431,7 @@ public void shouldRestoreOldUnversionedFormat() { -1, -1, "zxcv".getBytes(UTF_8), - zxcvRecordValue1), + hexStringToByteArray(zxcvBinaryValue1)), new ConsumerRecord<>("changelog-topic", 0, 3, @@ -446,7 +441,7 @@ public void shouldRestoreOldUnversionedFormat() { -1, -1, "zxcv".getBytes(UTF_8), - zxcvRecordValue2) + hexStringToByteArray(zxcvBinaryValue2)) )); assertThat(buffer.numRecords(), is(3)); @@ -514,33 +509,13 @@ public void shouldRestoreV1Format() { final RecordHeaders v1FlagHeaders = new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 1})}); - final byte[] todeleteValue = getContextualRecord("doomed", 0).serialize(0).array(); - - final byte[] asdfValue = getContextualRecord("qwer", 1).serialize(0).array(); - - final FullChangeSerde fullChangeSerde = FullChangeSerde.wrap(Serdes.String()); - - final byte[] zxcvValue1 = new ContextualRecord( - FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(fullChangeSerde.serializeParts(null, new Change<>("3o4im", "previous"))), - getContext(2L) - ).serialize(0).array(); - - final byte[] zxcvValue2 = new ContextualRecord( - FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(fullChangeSerde.serializeParts(null, new Change<>("next", "3o4im"))), - getContext(3L) - ).serialize(0).array(); - + // These serialized formats were captured by running version 2.2 code. + // They verify that an upgrade from 2.2 will work. + // Do not change them. final String toDeleteBinary = "00000000000000000000000000000000000000000000000000000005746F70696300000000FFFFFFFF0000000EFFFFFFFF00000006646F6F6D6564"; - final byte[] toDeleteArray = hexStringToByteArray(toDeleteBinary); - final String asdfBinary = "00000000000000020000000000000001000000000000000000000005746F70696300000000FFFFFFFF0000000CFFFFFFFF0000000471776572"; - final byte[] asdfArray = hexStringToByteArray(asdfBinary); - final String zxcvBinary1 = "00000000000000010000000000000002000000000000000000000005746F70696300000000FFFFFFFF000000150000000870726576696F757300000005336F34696D"; - final byte[] zxcvArray1 = hexStringToByteArray(zxcvBinary1); - final String zxcvBinary2 = "00000000000000010000000000000003000000000000000000000005746F70696300000000FFFFFFFF0000001100000005336F34696D000000046E657874"; - final byte[] zxcvArray2 = hexStringToByteArray(zxcvBinary2); stateRestoreCallback.restoreBatch(asList( new ConsumerRecord<>("changelog-topic", @@ -552,7 +527,7 @@ public void shouldRestoreV1Format() { -1, -1, "todelete".getBytes(UTF_8), - toDeleteArray, + hexStringToByteArray(toDeleteBinary), v1FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -563,7 +538,7 @@ public void shouldRestoreV1Format() { -1, -1, "asdf".getBytes(UTF_8), - asdfArray, + hexStringToByteArray(asdfBinary), v1FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -574,7 +549,7 @@ public void shouldRestoreV1Format() { -1, -1, "zxcv".getBytes(UTF_8), - zxcvArray1, + hexStringToByteArray(zxcvBinary1), v1FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -585,7 +560,7 @@ public void shouldRestoreV1Format() { -1, -1, "zxcv".getBytes(UTF_8), - zxcvArray2, + hexStringToByteArray(zxcvBinary2), v1FlagHeaders) )); @@ -641,6 +616,7 @@ public void shouldRestoreV1Format() { cleanup(context, buffer); } + @Test public void shouldRestoreV2Format() { final TimeOrderedKeyValueBuffer buffer = bufferSupplier.apply(testName); @@ -654,22 +630,14 @@ public void shouldRestoreV2Format() { final RecordHeaders v2FlagHeaders = new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 2})}); - final byte[] todeleteValue = getBufferValue("doomed", 0).serialize(0).array(); - final byte[] asdfValue = getBufferValue("qwer", 1).serialize(0).array(); - final byte[] zxcvValue1 = - new BufferValue( - Serdes.String().serializer().serialize(null, "previous"), - Serdes.String().serializer().serialize(null, "IGNORED"), - Serdes.String().serializer().serialize(null, "3o4im"), - getContext(2L) - ).serialize(0).array(); - final byte[] zxcvValue2 = - new BufferValue( - Serdes.String().serializer().serialize(null, "previous"), - Serdes.String().serializer().serialize(null, "3o4im"), - Serdes.String().serializer().serialize(null, "next"), - getContext(3L) - ).serialize(0).array(); + // These serialized formats were captured by running version 2.3 code. + // They verify that an upgrade from 2.3 will work. + // Do not change them. + final String toDeleteBinary = "0000000000000000000000000000000000000005746F70696300000000FFFFFFFF0000000EFFFFFFFF00000006646F6F6D6564FFFFFFFF0000000000000000"; + final String asdfBinary = "0000000000000001000000000000000000000005746F70696300000000FFFFFFFF0000000CFFFFFFFF0000000471776572FFFFFFFF0000000000000002"; + final String zxcvBinary1 = "0000000000000002000000000000000000000005746F70696300000000FFFFFFFF000000140000000749474E4F52454400000005336F34696D0000000870726576696F75730000000000000001"; + final String zxcvBinary2 = "0000000000000003000000000000000000000005746F70696300000000FFFFFFFF0000001100000005336F34696D000000046E6578740000000870726576696F75730000000000000001"; + stateRestoreCallback.restoreBatch(asList( new ConsumerRecord<>("changelog-topic", 0, @@ -680,7 +648,7 @@ public void shouldRestoreV2Format() { -1, -1, "todelete".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + todeleteValue.length).put(todeleteValue).putLong(0L).array(), + hexStringToByteArray(toDeleteBinary), v2FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -691,7 +659,7 @@ public void shouldRestoreV2Format() { -1, -1, "asdf".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + asdfValue.length).put(asdfValue).putLong(2L).array(), + hexStringToByteArray(asdfBinary), v2FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -702,7 +670,7 @@ public void shouldRestoreV2Format() { -1, -1, "zxcv".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + zxcvValue1.length).put(zxcvValue1).putLong(1L).array(), + hexStringToByteArray(zxcvBinary1), v2FlagHeaders), new ConsumerRecord<>("changelog-topic", 0, @@ -713,7 +681,7 @@ public void shouldRestoreV2Format() { -1, -1, "zxcv".getBytes(UTF_8), - ByteBuffer.allocate(Long.BYTES + zxcvValue2.length).put(zxcvValue2).putLong(1L).array(), + hexStringToByteArray(zxcvBinary2), v2FlagHeaders) )); @@ -769,6 +737,126 @@ public void shouldRestoreV2Format() { cleanup(context, buffer); } + @Test + public void shouldRestoreV3Format() { + final TimeOrderedKeyValueBuffer buffer = bufferSupplier.apply(testName); + final MockInternalProcessorContext context = makeContext(); + buffer.init(context, buffer); + + final RecordBatchingStateRestoreCallback stateRestoreCallback = + (RecordBatchingStateRestoreCallback) context.stateRestoreCallback(testName); + + context.setRecordContext(new ProcessorRecordContext(0, 0, 0, "", null)); + + final RecordHeaders headers = new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 3})}); + + // These serialized formats were captured by running version 2.4 code. + // They verify that an upgrade from 2.4 will work. + // Do not change them. + final String toDeleteBinary = "0000000000000000000000000000000000000005746F70696300000000FFFFFFFFFFFFFFFFFFFFFFFF00000006646F6F6D65640000000000000000"; + final String asdfBinary = "0000000000000001000000000000000000000005746F70696300000000FFFFFFFFFFFFFFFFFFFFFFFF00000004717765720000000000000002"; + final String zxcvBinary1 = "0000000000000002000000000000000000000005746F70696300000000FFFFFFFF0000000870726576696F75730000000749474E4F52454400000005336F34696D0000000000000001"; + final String zxcvBinary2 = "0000000000000003000000000000000000000005746F70696300000000FFFFFFFF0000000870726576696F757300000005336F34696D000000046E6578740000000000000001"; + + stateRestoreCallback.restoreBatch(asList( + new ConsumerRecord<>("changelog-topic", + 0, + 0, + 999, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "todelete".getBytes(UTF_8), + hexStringToByteArray(toDeleteBinary), + headers), + new ConsumerRecord<>("changelog-topic", + 0, + 1, + 9999, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "asdf".getBytes(UTF_8), + hexStringToByteArray(asdfBinary), + headers), + new ConsumerRecord<>("changelog-topic", + 0, + 2, + 99, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "zxcv".getBytes(UTF_8), + hexStringToByteArray(zxcvBinary1), + headers), + new ConsumerRecord<>("changelog-topic", + 0, + 2, + 100, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "zxcv".getBytes(UTF_8), + hexStringToByteArray(zxcvBinary2), + headers) + )); + + assertThat(buffer.numRecords(), is(3)); + assertThat(buffer.minTimestamp(), is(0L)); + assertThat(buffer.bufferSize(), is(142L)); + + stateRestoreCallback.restoreBatch(singletonList( + new ConsumerRecord<>("changelog-topic", + 0, + 3, + 3, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "todelete".getBytes(UTF_8), + null) + )); + + assertThat(buffer.numRecords(), is(2)); + assertThat(buffer.minTimestamp(), is(1L)); + assertThat(buffer.bufferSize(), is(95L)); + + assertThat(buffer.priorValueForBuffered("todelete"), is(Maybe.undefined())); + assertThat(buffer.priorValueForBuffered("asdf"), is(Maybe.defined(null))); + assertThat(buffer.priorValueForBuffered("zxcv"), is(Maybe.defined(ValueAndTimestamp.make("previous", -1)))); + + // flush the buffer into a list in buffer order so we can make assertions about the contents. + + final List> evicted = new LinkedList<>(); + buffer.evictWhile(() -> true, evicted::add); + + // Several things to note: + // * The buffered records are ordered according to their buffer time (serialized in the value of the changelog) + // * The record timestamps are properly restored, and not conflated with the record's buffer time. + // * The keys and values are properly restored + // * The record topic is set to the original input topic, *not* the changelog topic + // * The record offset preserves the original input record's offset, *not* the offset of the changelog record + + + assertThat(evicted, is(asList( + new Eviction<>( + "zxcv", + new Change<>("next", "3o4im"), + getContext(3L)), + new Eviction<>( + "asdf", + new Change<>("qwer", null), + getContext(1L) + )))); + + cleanup(context, buffer); + } + @Test public void shouldNotRestoreUnrecognizedVersionRecord() { final TimeOrderedKeyValueBuffer buffer = bufferSupplier.apply(testName); @@ -825,14 +913,6 @@ private static BufferValue getBufferValue(final String value, final long timesta ); } - private static ContextualRecord getContextualRecord(final String value, final long timestamp) { - final FullChangeSerde fullChangeSerde = FullChangeSerde.wrap(Serdes.String()); - return new ContextualRecord( - FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(fullChangeSerde.serializeParts(null, new Change<>(value, null))), - getContext(timestamp) - ); - } - private static ProcessorRecordContext getContext(final long recordTimestamp) { return new ProcessorRecordContext(recordTimestamp, 0, 0, "topic", null); } From e6e0b48a32945c2ddaa16da98148b5ca3b7634f5 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 15:31:19 -0500 Subject: [PATCH 14/23] cleanup --- .../kstream/internals/FullChangeSerde.java | 27 ----------- .../internals/FullChangeSerdeTest.java | 39 +++++++++++++--- .../TimeOrderedKeyValueBufferTest.java | 45 ++++++++++--------- 3 files changed, 57 insertions(+), 54 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java index 5d7c7e35697ee..5a4150a2887d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java @@ -68,33 +68,6 @@ public Change deserializeParts(final String topic, final Change seria return new Change<>(newValue, oldValue); } - /** - * We used to serialize a Change into a single byte[]. Now, we don't anymore, but we still keep this logic here - * so that we can produce the legacy format to test that we can still deserialize it. - */ - public static byte[] mergeChangeArraysIntoSingleLegacyFormattedArray(final Change serialChange) { - if (serialChange == null) { - return null; - } - - final int oldSize = serialChange.oldValue == null ? -1 : serialChange.oldValue.length; - final int newSize = serialChange.newValue == null ? -1 : serialChange.newValue.length; - - final ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES * 2 + Math.max(0, oldSize) + Math.max(0, newSize)); - - - buffer.putInt(oldSize); - if (serialChange.oldValue != null) { - buffer.put(serialChange.oldValue); - } - - buffer.putInt(newSize); - if (serialChange.newValue != null) { - buffer.put(serialChange.newValue); - } - return buffer.array(); - } - /** * We used to serialize a Change into a single byte[]. Now, we don't anymore, but we still * need to be able to read it (so that we can load the state store from previously-written changelog records). diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FullChangeSerdeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FullChangeSerdeTest.java index ac6762f504cf7..e7e0c88bd1eef 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FullChangeSerdeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FullChangeSerdeTest.java @@ -19,6 +19,8 @@ import org.apache.kafka.common.serialization.Serdes; import org.junit.Test; +import java.nio.ByteBuffer; + import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.Is.is; @@ -26,10 +28,37 @@ public class FullChangeSerdeTest { private final FullChangeSerde serde = FullChangeSerde.wrap(Serdes.String()); + /** + * We used to serialize a Change into a single byte[]. Now, we don't anymore, but we still keep this logic here + * so that we can produce the legacy format to test that we can still deserialize it. + */ + private static byte[] mergeChangeArraysIntoSingleLegacyFormattedArray(final Change serialChange) { + if (serialChange == null) { + return null; + } + + final int oldSize = serialChange.oldValue == null ? -1 : serialChange.oldValue.length; + final int newSize = serialChange.newValue == null ? -1 : serialChange.newValue.length; + + final ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES * 2 + Math.max(0, oldSize) + Math.max(0, newSize)); + + + buffer.putInt(oldSize); + if (serialChange.oldValue != null) { + buffer.put(serialChange.oldValue); + } + + buffer.putInt(newSize); + if (serialChange.newValue != null) { + buffer.put(serialChange.newValue); + } + return buffer.array(); + } + @Test public void shouldRoundTripNull() { assertThat(serde.serializeParts(null, null), nullValue()); - assertThat(FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(null), nullValue()); + assertThat(mergeChangeArraysIntoSingleLegacyFormattedArray(null), nullValue()); assertThat(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(null), nullValue()); assertThat(serde.deserializeParts(null, null), nullValue()); } @@ -47,7 +76,7 @@ public void shouldRoundTripNullChange() { is(new Change(null, null)) ); - final byte[] legacyFormat = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(new Change<>(null, null)); + final byte[] legacyFormat = mergeChangeArraysIntoSingleLegacyFormattedArray(new Change<>(null, null)); assertThat( FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(legacyFormat), is(new Change(null, null)) @@ -57,7 +86,7 @@ public void shouldRoundTripNullChange() { @Test public void shouldRoundTripOldNull() { final Change serialized = serde.serializeParts(null, new Change<>("new", null)); - final byte[] legacyFormat = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(serialized); + final byte[] legacyFormat = mergeChangeArraysIntoSingleLegacyFormattedArray(serialized); final Change decomposedLegacyFormat = FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(legacyFormat); assertThat( serde.deserializeParts(null, decomposedLegacyFormat), @@ -68,7 +97,7 @@ public void shouldRoundTripOldNull() { @Test public void shouldRoundTripNewNull() { final Change serialized = serde.serializeParts(null, new Change<>(null, "old")); - final byte[] legacyFormat = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(serialized); + final byte[] legacyFormat = mergeChangeArraysIntoSingleLegacyFormattedArray(serialized); final Change decomposedLegacyFormat = FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(legacyFormat); assertThat( serde.deserializeParts(null, decomposedLegacyFormat), @@ -79,7 +108,7 @@ public void shouldRoundTripNewNull() { @Test public void shouldRoundTripChange() { final Change serialized = serde.serializeParts(null, new Change<>("new", "old")); - final byte[] legacyFormat = FullChangeSerde.mergeChangeArraysIntoSingleLegacyFormattedArray(serialized); + final byte[] legacyFormat = mergeChangeArraysIntoSingleLegacyFormattedArray(serialized); final Change decomposedLegacyFormat = FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(legacyFormat); assertThat( serde.deserializeParts(null, decomposedLegacyFormat), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 9b8e50680d8d2..20805852a2642 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -360,28 +360,6 @@ public void shouldFlush() { cleanup(context, buffer); } - private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); - - private static String bytesToHex(final byte[] bytes) { - final char[] hexChars = new char[bytes.length * 2]; - for (int j = 0; j < bytes.length; j++) { - final int v = bytes[j] & 0xFF; - hexChars[j * 2] = HEX_ARRAY[v >>> 4]; - hexChars[j * 2 + 1] = HEX_ARRAY[v & 0x0F]; - } - return new String(hexChars); - } - - private static byte[] hexStringToByteArray(final String hexString) { - final int len = hexString.length(); - final byte[] data = new byte[len / 2]; - for (int i = 0; i < len; i += 2) { - data[i / 2] = (byte) ((Character.digit(hexString.charAt(i), 16) << 4) - + Character.digit(hexString.charAt(i+1), 16)); - } - return data; - } - @Test public void shouldRestoreOldUnversionedFormat() { final TimeOrderedKeyValueBuffer buffer = bufferSupplier.apply(testName); @@ -916,4 +894,27 @@ private static BufferValue getBufferValue(final String value, final long timesta private static ProcessorRecordContext getContext(final long recordTimestamp) { return new ProcessorRecordContext(recordTimestamp, 0, 0, "topic", null); } + + + // to be used to generate future hex-encoded values +// private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); +// private static String bytesToHex(final byte[] bytes) { +// final char[] hexChars = new char[bytes.length * 2]; +// for (int j = 0; j < bytes.length; j++) { +// final int v = bytes[j] & 0xFF; +// hexChars[j * 2] = HEX_ARRAY[v >>> 4]; +// hexChars[j * 2 + 1] = HEX_ARRAY[v & 0x0F]; +// } +// return new String(hexChars); +// } + + private static byte[] hexStringToByteArray(final String hexString) { + final int len = hexString.length(); + final byte[] data = new byte[len / 2]; + for (int i = 0; i < len; i += 2) { + data[i / 2] = (byte) ((Character.digit(hexString.charAt(i), 16) << 4) + + Character.digit(hexString.charAt(i + 1), 16)); + } + return data; + } } From 65a549f98d27010e43d07154b3c30e3a9e41f318 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 21:00:45 -0500 Subject: [PATCH 15/23] adding other old versions --- .../kafka/streams/tests/SmokeTestClient.java | 298 +++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 622 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 134 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 99 +++ .../kafka/streams/tests/SmokeTestClient.java | 298 +++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 622 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 134 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 99 +++ .../kafka/streams/tests/SmokeTestClient.java | 298 +++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 622 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 134 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 99 +++ .../kafka/streams/tests/SmokeTestClient.java | 298 +++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 622 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 134 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 99 +++ 16 files changed, 4612 insertions(+) create mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java create mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java create mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000000..ced1369af3855 --- /dev/null +++ b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler((t, e) -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + e.printStackTrace(System.out); + uncaughtException = true; + streams.close(Duration.ofSeconds(30)); + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) + .reduce((l, r) -> l + r); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000000..ac83cd95ebaaf --- /dev/null +++ b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,622 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + List> needRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, needRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor("data"); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + )); + } + } + return Collections.unmodifiableMap(allData); + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000000..e8ec04c23ca52 --- /dev/null +++ b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.time.Instant; + +public class SmokeTestUtil { + + final static int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return new ProcessorSupplier() { + @Override + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Object key, final Object value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + if (smallestOffset > context().offset()) { + smallestOffset = context().offset(); + } + if (largestOffset < context().offset()) { + largestOffset = context().offset(); + } + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000000..07c7d5d82ebff --- /dev/null +++ b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + System.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + System.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE); + + System.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000000..ced1369af3855 --- /dev/null +++ b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler((t, e) -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + e.printStackTrace(System.out); + uncaughtException = true; + streams.close(Duration.ofSeconds(30)); + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) + .reduce((l, r) -> l + r); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000000..ac83cd95ebaaf --- /dev/null +++ b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,622 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + List> needRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, needRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor("data"); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + )); + } + } + return Collections.unmodifiableMap(allData); + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000000..e8ec04c23ca52 --- /dev/null +++ b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.time.Instant; + +public class SmokeTestUtil { + + final static int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return new ProcessorSupplier() { + @Override + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Object key, final Object value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + if (smallestOffset > context().offset()) { + smallestOffset = context().offset(); + } + if (largestOffset < context().offset()) { + largestOffset = context().offset(); + } + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000000..07c7d5d82ebff --- /dev/null +++ b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + System.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + System.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE); + + System.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000000..ced1369af3855 --- /dev/null +++ b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler((t, e) -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + e.printStackTrace(System.out); + uncaughtException = true; + streams.close(Duration.ofSeconds(30)); + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) + .reduce((l, r) -> l + r); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000000..ac83cd95ebaaf --- /dev/null +++ b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,622 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + List> needRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, needRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor("data"); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + )); + } + } + return Collections.unmodifiableMap(allData); + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000000..e8ec04c23ca52 --- /dev/null +++ b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.time.Instant; + +public class SmokeTestUtil { + + final static int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return new ProcessorSupplier() { + @Override + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Object key, final Object value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + if (smallestOffset > context().offset()) { + smallestOffset = context().offset(); + } + if (largestOffset < context().offset()) { + largestOffset = context().offset(); + } + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000000..07c7d5d82ebff --- /dev/null +++ b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + System.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + System.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE); + + System.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000000..ced1369af3855 --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler((t, e) -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + e.printStackTrace(System.out); + uncaughtException = true; + streams.close(Duration.ofSeconds(30)); + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) + .reduce((l, r) -> l + r); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.of(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000000..ac83cd95ebaaf --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,622 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + List> needRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, needRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor("data"); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + )); + } + } + return Collections.unmodifiableMap(allData); + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000000..e8ec04c23ca52 --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.time.Instant; + +public class SmokeTestUtil { + + final static int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return new ProcessorSupplier() { + @Override + public Processor get() { + return new AbstractProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Object key, final Object value) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + if (smallestOffset > context().offset()) { + smallestOffset = context().offset(); + } + if (largestOffset < context().offset()) { + largestOffset = context().offset(); + } + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000000..07c7d5d82ebff --- /dev/null +++ b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + System.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + System.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE); + + System.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} From 41a4db1a7c5bd2bb41a861e6f7889ddd9bf062a4 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 21:01:08 -0500 Subject: [PATCH 16/23] fallback to make 2.4 work --- .../InMemoryTimeOrderedKeyValueBuffer.java | 83 +++++++++++++++---- .../kafka/streams/tests/SmokeTestDriver.java | 10 +++ .../kafka/streams/tests/SmokeTestDriver.java | 10 +++ .../tests/streams/streams_upgrade_test.py | 5 +- 4 files changed, 90 insertions(+), 18 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 7d3798946d72c..b15f9c43cec17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -363,27 +363,37 @@ private void restoreBatch(final Collection> batch ) ); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_2_CHANGELOG_HEADER_VALUE)) { - final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); - final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); - final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); + DeserializationResult deserializationResult = null; + RuntimeException v2DeserializationException = null; + RuntimeException v3DeserializationException = null; + try { + deserializationResult = getDeserializationResultV2(record, key); + } catch (final RuntimeException e) { + v2DeserializationException = e; + } + // versions 2.4.0, 2.4.1, and 2.5.0 would have erroneously encoded a V3 record with the + // V2 header, so we'll try duck-typing to see if this is decodable as V3 + if (deserializationResult == null) { + try { + deserializationResult = getDeserializationResultV3(record, key); + } catch (final RuntimeException e) { + v3DeserializationException = e; + } + } - final int priorValueLength = valueAndTime.getInt(); - final byte[] priorValue; - if (priorValueLength == -1) { - priorValue = null; + if (deserializationResult == null) { + // ok, it wasn't V3 either. Throw both exceptions: + final RuntimeException exception = + new RuntimeException("Couldn't deserialize record as v2 or v3: " + record, + v2DeserializationException); + exception.addSuppressed(v3DeserializationException); + throw exception; } else { - priorValue = new byte[priorValueLength]; - valueAndTime.get(priorValue); + cleanPut(deserializationResult.time, deserializationResult.key, deserializationResult.bufferValue); } - final long time = valueAndTime.getLong(); - final BufferValue bufferValue = new BufferValue(priorValue, change.oldValue, change.newValue, contextualRecord.recordContext()); - cleanPut(time, key, bufferValue); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_3_CHANGELOG_HEADER_VALUE)) { - - final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); - final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); - final long time = valueAndTime.getLong(); - cleanPut(time, key, bufferValue); + final DeserializationResult deserializationResult = getDeserializationResultV3(record, key); + cleanPut(deserializationResult.time, deserializationResult.key, deserializationResult.bufferValue); } else { throw new IllegalArgumentException("Restoring apparently invalid changelog record: " + record); } @@ -392,6 +402,45 @@ private void restoreBatch(final Collection> batch updateBufferMetrics(); } + private InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV3(final ConsumerRecord record, final Bytes key) { + final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); + final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); + final long time = valueAndTime.getLong(); + return new DeserializationResult(time, key, bufferValue); + } + + private InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV2(final ConsumerRecord record, final Bytes key) { + final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); + final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); + final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); + + final int priorValueLength = valueAndTime.getInt(); + final byte[] priorValue; + if (priorValueLength == -1) { + priorValue = null; + } else { + priorValue = new byte[priorValueLength]; + valueAndTime.get(priorValue); + } + final long time = valueAndTime.getLong(); + final BufferValue bufferValue = new BufferValue(priorValue, change.oldValue, change.newValue, contextualRecord.recordContext()); + return new DeserializationResult(time, key, bufferValue); + } + + private static final class DeserializationResult { + + private final long time; + private final Bytes key; + private final BufferValue bufferValue; + + public DeserializationResult(final long time, final Bytes key, final BufferValue bufferValue) { + + this.time = time; + this.key = key; + this.bufferValue = bufferValue; + } + } + @Override public void evictWhile(final Supplier predicate, final Consumer> callback) { diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java index ac83cd95ebaaf..d0a7d22d938f8 100644 --- a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -274,6 +274,11 @@ private static void shuffle(final int[] data, @SuppressWarnings("SameParameterVa } public static class NumberDeserializer implements Deserializer { + @Override + public void configure(final Map configs, final boolean isKey) { + + } + @Override public Number deserialize(final String topic, final byte[] data) { final Number value; @@ -302,6 +307,11 @@ public Number deserialize(final String topic, final byte[] data) { } return value; } + + @Override + public void close() { + + } } public static VerificationResult verify(final String kafka, diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java index ac83cd95ebaaf..d0a7d22d938f8 100644 --- a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -274,6 +274,11 @@ private static void shuffle(final int[] data, @SuppressWarnings("SameParameterVa } public static class NumberDeserializer implements Deserializer { + @Override + public void configure(final Map configs, final boolean isKey) { + + } + @Override public Number deserialize(final String topic, final byte[] data) { final Number value; @@ -302,6 +307,11 @@ public Number deserialize(final String topic, final byte[] data) { } return value; } + + @Override + public void close() { + + } } public static VerificationResult verify(final String kafka, diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 6f66d165e5e0a..862bd34350304 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -37,6 +37,7 @@ # can be replaced with metadata_2_versions backward_compatible_metadata_2_versions = [str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)] metadata_3_or_higher_versions = [str(LATEST_2_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(DEV_VERSION)] +smoke_test_versions = [str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(DEV_VERSION)] dev_version = [str(DEV_VERSION)] @@ -244,7 +245,7 @@ def test_metadata_upgrade(self, from_version, to_version): timeout_sec=60, err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) - @matrix(from_version=metadata_3_or_higher_versions, to_version=dev_version) + @matrix(from_version=smoke_test_versions, to_version=dev_version) def test_app_upgrade(self, from_version, to_version): """ Starts 3 KafkaStreams instances with , and upgrades one-by-one to @@ -298,6 +299,8 @@ def test_app_upgrade(self, from_version, to_version): # shutdown self.driver.stop() + # ideally, we would actually verify the expected results, but it + random.shuffle(self.processors) for p in self.processors: node = p.node From 63f1cc3ab18d301c085aeaecad98879a4d85e49b Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 22:07:58 -0500 Subject: [PATCH 17/23] remove 2.1. cf KAFKA-10203 --- .../InMemoryTimeOrderedKeyValueBuffer.java | 38 +- .../TimeOrderedKeyValueBufferTest.java | 123 ++++ .../kafka/streams/tests/SmokeTestClient.java | 298 --------- .../kafka/streams/tests/SmokeTestDriver.java | 632 ------------------ .../kafka/streams/tests/SmokeTestUtil.java | 134 ---- .../kafka/streams/tests/StreamsSmokeTest.java | 99 --- .../tests/streams/streams_upgrade_test.py | 5 +- 7 files changed, 146 insertions(+), 1183 deletions(-) delete mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java delete mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java delete mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java delete mode 100644 streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index b15f9c43cec17..89e8d049c9173 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -262,27 +262,27 @@ private void logValue(final Bytes key, final BufferKey bufferKey, final BufferVa buffer.putLong(bufferKey.time()); final byte[] array = buffer.array(); ((RecordCollector.Supplier) context).recordCollector().send( - changelogTopic, - key, - array, - CHANGELOG_HEADERS, - partition, - null, - KEY_SERIALIZER, - VALUE_SERIALIZER + changelogTopic, + key, + array, + CHANGELOG_HEADERS, + partition, + null, + KEY_SERIALIZER, + VALUE_SERIALIZER ); } private void logTombstone(final Bytes key) { ((RecordCollector.Supplier) context).recordCollector().send( - changelogTopic, - key, - null, - null, - partition, - null, - KEY_SERIALIZER, - VALUE_SERIALIZER + changelogTopic, + key, + null, + null, + partition, + null, + KEY_SERIALIZER, + VALUE_SERIALIZER ); } @@ -402,14 +402,16 @@ private void restoreBatch(final Collection> batch updateBufferMetrics(); } - private InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV3(final ConsumerRecord record, final Bytes key) { + private static InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV3(final ConsumerRecord record, + final Bytes key) { final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); final long time = valueAndTime.getLong(); return new DeserializationResult(time, key, bufferValue); } - private InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV2(final ConsumerRecord record, final Bytes key) { + private static InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV2(final ConsumerRecord record, + final Bytes key) { final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 20805852a2642..8c4904a6fd680 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -715,6 +715,129 @@ public void shouldRestoreV2Format() { cleanup(context, buffer); } + @Test + public void shouldRestoreV3FormatWithV2Header() { + // versions 2.4.0, 2.4.1, and 2.5.0 would have erroneously encoded a V3 record with the + // V2 header, so we need to be sure to handle this case as well. + // Note the data is the same as the V3 test. + final TimeOrderedKeyValueBuffer buffer = bufferSupplier.apply(testName); + final MockInternalProcessorContext context = makeContext(); + buffer.init(context, buffer); + + final RecordBatchingStateRestoreCallback stateRestoreCallback = + (RecordBatchingStateRestoreCallback) context.stateRestoreCallback(testName); + + context.setRecordContext(new ProcessorRecordContext(0, 0, 0, "", null)); + + final RecordHeaders headers = new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 2})}); + + // These serialized formats were captured by running version 2.4 code. + // They verify that an upgrade from 2.4 will work. + // Do not change them. + final String toDeleteBinary = "0000000000000000000000000000000000000005746F70696300000000FFFFFFFFFFFFFFFFFFFFFFFF00000006646F6F6D65640000000000000000"; + final String asdfBinary = "0000000000000001000000000000000000000005746F70696300000000FFFFFFFFFFFFFFFFFFFFFFFF00000004717765720000000000000002"; + final String zxcvBinary1 = "0000000000000002000000000000000000000005746F70696300000000FFFFFFFF0000000870726576696F75730000000749474E4F52454400000005336F34696D0000000000000001"; + final String zxcvBinary2 = "0000000000000003000000000000000000000005746F70696300000000FFFFFFFF0000000870726576696F757300000005336F34696D000000046E6578740000000000000001"; + + stateRestoreCallback.restoreBatch(asList( + new ConsumerRecord<>("changelog-topic", + 0, + 0, + 999, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "todelete".getBytes(UTF_8), + hexStringToByteArray(toDeleteBinary), + headers), + new ConsumerRecord<>("changelog-topic", + 0, + 1, + 9999, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "asdf".getBytes(UTF_8), + hexStringToByteArray(asdfBinary), + headers), + new ConsumerRecord<>("changelog-topic", + 0, + 2, + 99, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "zxcv".getBytes(UTF_8), + hexStringToByteArray(zxcvBinary1), + headers), + new ConsumerRecord<>("changelog-topic", + 0, + 2, + 100, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "zxcv".getBytes(UTF_8), + hexStringToByteArray(zxcvBinary2), + headers) + )); + + assertThat(buffer.numRecords(), is(3)); + assertThat(buffer.minTimestamp(), is(0L)); + assertThat(buffer.bufferSize(), is(142L)); + + stateRestoreCallback.restoreBatch(singletonList( + new ConsumerRecord<>("changelog-topic", + 0, + 3, + 3, + TimestampType.CREATE_TIME, + -1L, + -1, + -1, + "todelete".getBytes(UTF_8), + null) + )); + + assertThat(buffer.numRecords(), is(2)); + assertThat(buffer.minTimestamp(), is(1L)); + assertThat(buffer.bufferSize(), is(95L)); + + assertThat(buffer.priorValueForBuffered("todelete"), is(Maybe.undefined())); + assertThat(buffer.priorValueForBuffered("asdf"), is(Maybe.defined(null))); + assertThat(buffer.priorValueForBuffered("zxcv"), is(Maybe.defined(ValueAndTimestamp.make("previous", -1)))); + + // flush the buffer into a list in buffer order so we can make assertions about the contents. + + final List> evicted = new LinkedList<>(); + buffer.evictWhile(() -> true, evicted::add); + + // Several things to note: + // * The buffered records are ordered according to their buffer time (serialized in the value of the changelog) + // * The record timestamps are properly restored, and not conflated with the record's buffer time. + // * The keys and values are properly restored + // * The record topic is set to the original input topic, *not* the changelog topic + // * The record offset preserves the original input record's offset, *not* the offset of the changelog record + + + assertThat(evicted, is(asList( + new Eviction<>( + "zxcv", + new Change<>("next", "3o4im"), + getContext(3L)), + new Eviction<>( + "asdf", + new Change<>("qwer", null), + getContext(1L) + )))); + + cleanup(context, buffer); + } + @Test public void shouldRestoreV3Format() { final TimeOrderedKeyValueBuffer buffer = bufferSupplier.apply(testName); diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java deleted file mode 100644 index ced1369af3855..0000000000000 --- a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ /dev/null @@ -1,298 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.Topology; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Grouped; -import org.apache.kafka.streams.kstream.KGroupedStream; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; -import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.streams.state.WindowStore; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; - -public class SmokeTestClient extends SmokeTestUtil { - - private final String name; - - private KafkaStreams streams; - private boolean uncaughtException = false; - private boolean started; - private volatile boolean closed; - - private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { - Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); - } else { - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } - } - - private static File tempDirectory() { - final String prefix = "kafka-"; - final File file; - try { - file = Files.createTempDirectory(prefix).toFile(); - } catch (final IOException ex) { - throw new RuntimeException("Failed to create a temp dir", ex); - } - file.deleteOnExit(); - - addShutdownHook("delete-temp-file-shutdown-hook", () -> { - try { - Utils.delete(file); - } catch (final IOException e) { - System.out.println("Error deleting " + file.getAbsolutePath()); - e.printStackTrace(System.out); - } - }); - - return file; - } - - public SmokeTestClient(final String name) { - this.name = name; - } - - public boolean started() { - return started; - } - - public boolean closed() { - return closed; - } - - public void start(final Properties streamsProperties) { - final Topology build = getTopology(); - streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); - - final CountDownLatch countDownLatch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; - countDownLatch.countDown(); - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - - streams.setUncaughtExceptionHandler((t, e) -> { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - e.printStackTrace(System.out); - uncaughtException = true; - streams.close(Duration.ofSeconds(30)); - }); - - addShutdownHook("streams-shutdown-hook", this::close); - - streams.start(); - try { - if (!countDownLatch.await(1, TimeUnit.MINUTES)) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); - } - } catch (final InterruptedException e) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); - e.printStackTrace(System.out); - } - System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); - System.out.println(name + " started at " + Instant.now()); - } - - public void closeAsync() { - streams.close(Duration.ZERO); - } - - public void close() { - final boolean closed = streams.close(Duration.ofMinutes(1)); - - if (closed && !uncaughtException) { - System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } else if (closed) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - } else { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); - } - } - - private Properties getStreamsConfig(final Properties props) { - final Properties fullProps = new Properties(props); - fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); - fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); - fullProps.putAll(props); - return fullProps; - } - - public Topology getTopology() { - final StreamsBuilder builder = new StreamsBuilder(); - final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); - final KStream source = builder.stream("data", stringIntConsumed); - source.filterNot((k, v) -> k.equals("flush")) - .to("echo", Produced.with(stringSerde, intSerde)); - final KStream data = source.filter((key, value) -> value == null || value != END); - data.process(SmokeTestUtil.printProcessorSupplier("data", name)); - - // min - final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); - - final KTable, Integer> minAggregation = groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) - .aggregate( - () -> Integer.MAX_VALUE, - (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, - Materialized - .>as("uwin-min") - .withValueSerde(intSerde) - .withRetention(Duration.ofHours(25)) - ); - - streamify(minAggregation, "min-raw"); - - streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); - - minAggregation - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("min", Produced.with(stringSerde, intSerde)); - - final KTable, Integer> smallWindowSum = groupedData - .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) - .reduce((l, r) -> l + r); - - streamify(smallWindowSum, "sws-raw"); - streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); - - final KTable minTable = builder.table( - "min", - Consumed.with(stringSerde, intSerde), - Materialized.as("minStoreName")); - - minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); - - // max - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> Integer.MIN_VALUE, - (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, - Materialized.>as("uwin-max").withValueSerde(intSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("max", Produced.with(stringSerde, intSerde)); - - final KTable maxTable = builder.table( - "max", - Consumed.with(stringSerde, intSerde), - Materialized.as("maxStoreName")); - maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); - - // sum - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> 0L, - (aggKey, value, aggregate) -> (long) value + aggregate, - Materialized.>as("win-sum").withValueSerde(longSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("sum", Produced.with(stringSerde, longSerde)); - - final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); - final KTable sumTable = builder.table("sum", stringLongConsumed); - sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); - - // cnt - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .count(Materialized.as("uwin-cnt")) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("cnt", Produced.with(stringSerde, longSerde)); - - final KTable cntTable = builder.table( - "cnt", - Consumed.with(stringSerde, longSerde), - Materialized.as("cntStoreName")); - cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); - - // dif - maxTable - .join( - minTable, - (value1, value2) -> value1 - value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("dif", Produced.with(stringSerde, intSerde)); - - // avg - sumTable - .join( - cntTable, - (value1, value2) -> (double) value1 / (double) value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("avg", Produced.with(stringSerde, doubleSerde)); - - // test repartition - final Agg agg = new Agg(); - cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) - .aggregate(agg.init(), agg.adder(), agg.remover(), - Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) - .withKeySerde(Serdes.String()) - .withValueSerde(Serdes.Long())) - .toStream() - .to("tagg", Produced.with(stringSerde, longSerde)); - - return builder.build(); - } - - private static void streamify(final KTable, Integer> windowedTable, final String topic) { - windowedTable - .toStream() - .filterNot((k, v) -> k.key().equals("flush")) - .map((key, value) -> new KeyValue<>(key.toString(), value)) - .to(topic, Produced.with(stringSerde, intSerde)); - } -} diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java deleted file mode 100644 index d0a7d22d938f8..0000000000000 --- a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ /dev/null @@ -1,632 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.utils.Utils; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; - -public class SmokeTestDriver extends SmokeTestUtil { - private static final String[] TOPICS = { - "data", - "echo", - "max", - "min", "min-suppressed", "min-raw", - "dif", - "sum", - "sws-raw", "sws-suppressed", - "cnt", - "avg", - "tagg" - }; - - private static final int MAX_RECORD_EMPTY_RETRIES = 30; - - private static class ValueList { - public final String key; - private final int[] values; - private int index; - - ValueList(final int min, final int max) { - key = min + "-" + max; - - values = new int[max - min + 1]; - for (int i = 0; i < values.length; i++) { - values[i] = min + i; - } - // We want to randomize the order of data to test not completely predictable processing order - // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) - // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window - shuffle(values, 10); - - index = 0; - } - - int next() { - return (index < values.length) ? values[index++] : -1; - } - } - - public static String[] topics() { - return Arrays.copyOf(TOPICS, TOPICS.length); - } - - static void generatePerpetually(final String kafka, - final int numKeys, - final int maxRecordsPerKey) { - final Properties producerProps = generatorProperties(kafka); - - int numRecordsProduced = 0; - - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - } - - final Random rand = new Random(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (true) { - final int index = rand.nextInt(numKeys); - final String key = data[index].key; - final int value = data[index].next(); - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record); - - numRecordsProduced++; - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(2); - } - } - } - - public static Map> generate(final String kafka, - final int numKeys, - final int maxRecordsPerKey, - final Duration timeToSpend) { - final Properties producerProps = generatorProperties(kafka); - - - int numRecordsProduced = 0; - - final Map> allData = new HashMap<>(); - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - allData.put(data[i].key, new HashSet<>()); - } - final Random rand = new Random(); - - int remaining = data.length; - - final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; - - List> needRetry = new ArrayList<>(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (remaining > 0) { - final int index = rand.nextInt(remaining); - final String key = data[index].key; - final int value = data[index].next(); - - if (value < 0) { - remaining--; - data[index] = data[remaining]; - } else { - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record, new TestCallback(record, needRetry)); - - numRecordsProduced++; - allData.get(key).add(value); - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(Math.max(recordPauseTime, 2)); - } - } - producer.flush(); - - int remainingRetries = 5; - while (!needRetry.isEmpty()) { - final List> needRetry2 = new ArrayList<>(); - for (final ProducerRecord record : needRetry) { - System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); - producer.send(record, new TestCallback(record, needRetry2)); - } - producer.flush(); - needRetry = needRetry2; - - if (--remainingRetries == 0 && !needRetry.isEmpty()) { - System.err.println("Failed to produce all records after multiple retries"); - Exit.exit(1); - } - } - - // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out - // all suppressed records. - final List partitions = producer.partitionsFor("data"); - for (final PartitionInfo partition : partitions) { - producer.send(new ProducerRecord<>( - partition.topic(), - partition.partition(), - System.currentTimeMillis() + Duration.ofDays(2).toMillis(), - stringSerde.serializer().serialize("", "flush"), - intSerde.serializer().serialize("", 0) - )); - } - } - return Collections.unmodifiableMap(allData); - } - - private static Properties generatorProperties(final String kafka) { - final Properties producerProps = new Properties(); - producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); - return producerProps; - } - - private static class TestCallback implements Callback { - private final ProducerRecord originalRecord; - private final List> needRetry; - - TestCallback(final ProducerRecord originalRecord, - final List> needRetry) { - this.originalRecord = originalRecord; - this.needRetry = needRetry; - } - - @Override - public void onCompletion(final RecordMetadata metadata, final Exception exception) { - if (exception != null) { - if (exception instanceof TimeoutException) { - needRetry.add(originalRecord); - } else { - exception.printStackTrace(); - Exit.exit(1); - } - } - } - } - - private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { - final Random rand = new Random(); - for (int i = 0; i < data.length; i++) { - // we shuffle data within windowSize - final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; - - // swap - final int tmp = data[i]; - data[i] = data[j]; - data[j] = tmp; - } - } - - public static class NumberDeserializer implements Deserializer { - @Override - public void configure(final Map configs, final boolean isKey) { - - } - - @Override - public Number deserialize(final String topic, final byte[] data) { - final Number value; - switch (topic) { - case "data": - case "echo": - case "min": - case "min-raw": - case "min-suppressed": - case "sws-raw": - case "sws-suppressed": - case "max": - case "dif": - value = intSerde.deserializer().deserialize(topic, data); - break; - case "sum": - case "cnt": - case "tagg": - value = longSerde.deserializer().deserialize(topic, data); - break; - case "avg": - value = doubleSerde.deserializer().deserialize(topic, data); - break; - default: - throw new RuntimeException("unknown topic: " + topic); - } - return value; - } - - @Override - public void close() { - - } - } - - public static VerificationResult verify(final String kafka, - final Map> inputs, - final int maxRecordsPerKey) { - final Properties props = new Properties(); - props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); - props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); - - final KafkaConsumer consumer = new KafkaConsumer<>(props); - final List partitions = getAllPartitions(consumer, TOPICS); - consumer.assign(partitions); - consumer.seekToBeginning(partitions); - - final int recordsGenerated = inputs.size() * maxRecordsPerKey; - int recordsProcessed = 0; - final Map processed = - Stream.of(TOPICS) - .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); - - final Map>>> events = new HashMap<>(); - - VerificationResult verificationResult = new VerificationResult(false, "no results yet"); - int retry = 0; - final long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { - final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (records.isEmpty() && recordsProcessed >= recordsGenerated) { - verificationResult = verifyAll(inputs, events, false); - if (verificationResult.passed()) { - break; - } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { - System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); - break; - } else { - System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); - } - } else { - System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); - - retry = 0; - for (final ConsumerRecord record : records) { - final String key = record.key(); - - final String topic = record.topic(); - processed.get(topic).incrementAndGet(); - - if (topic.equals("echo")) { - recordsProcessed++; - if (recordsProcessed % 100 == 0) { - System.out.println("Echo records processed = " + recordsProcessed); - } - } - - events.computeIfAbsent(topic, t -> new HashMap<>()) - .computeIfAbsent(key, k -> new LinkedList<>()) - .add(record); - } - - System.out.println(processed); - } - } - consumer.close(); - final long finished = System.currentTimeMillis() - start; - System.out.println("Verification time=" + finished); - System.out.println("-------------------"); - System.out.println("Result Verification"); - System.out.println("-------------------"); - System.out.println("recordGenerated=" + recordsGenerated); - System.out.println("recordProcessed=" + recordsProcessed); - - if (recordsProcessed > recordsGenerated) { - System.out.println("PROCESSED-MORE-THAN-GENERATED"); - } else if (recordsProcessed < recordsGenerated) { - System.out.println("PROCESSED-LESS-THAN-GENERATED"); - } - - boolean success; - - final Map> received = - events.get("echo") - .entrySet() - .stream() - .map(entry -> mkEntry( - entry.getKey(), - entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) - ) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - success = inputs.equals(received); - - if (success) { - System.out.println("ALL-RECORDS-DELIVERED"); - } else { - int missedCount = 0; - for (final Map.Entry> entry : inputs.entrySet()) { - missedCount += received.get(entry.getKey()).size(); - } - System.out.println("missedRecords=" + missedCount); - } - - // give it one more try if it's not already passing. - if (!verificationResult.passed()) { - verificationResult = verifyAll(inputs, events, true); - } - success &= verificationResult.passed(); - - System.out.println(verificationResult.result()); - - System.out.println(success ? "SUCCESS" : "FAILURE"); - return verificationResult; - } - - public static class VerificationResult { - private final boolean passed; - private final String result; - - VerificationResult(final boolean passed, final String result) { - this.passed = passed; - this.result = result; - } - - public boolean passed() { - return passed; - } - - public String result() { - return result; - } - } - - private static VerificationResult verifyAll(final Map> inputs, - final Map>>> events, - final boolean printResults) { - final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - boolean pass; - try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { - pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); - pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); - pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { - final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); - return getMin(unwindowedKey); - }, printResults); - pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); - pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); - pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); - pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); - pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); - pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); - pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); - } - return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); - } - - private static boolean verify(final PrintStream resultStream, - final String topic, - final Map> inputData, - final Map>>> events, - final Function keyToExpectation, - final boolean printResults) { - final Map>> observedInputEvents = events.get("data"); - final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); - if (outputEvents.isEmpty()) { - resultStream.println(topic + " is empty"); - return false; - } else { - resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); - - if (outputEvents.size() != inputData.size()) { - resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", - outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); - return false; - } - for (final Map.Entry>> entry : outputEvents.entrySet()) { - final String key = entry.getKey(); - final Number expected = keyToExpectation.apply(key); - final Number actual = entry.getValue().getLast().value(); - if (!expected.equals(actual)) { - resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); - - if (printResults) { - resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); - - if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) - resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); - } - - return false; - } - } - return true; - } - } - - - private static boolean verifySuppressed(final PrintStream resultStream, - @SuppressWarnings("SameParameterValue") final String topic, - final Map>>> events, - final boolean printResults) { - resultStream.println("verifying suppressed " + topic); - final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); - for (final Map.Entry>> entry : topicEvents.entrySet()) { - if (entry.getValue().size() != 1) { - final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); - final String key = entry.getKey(); - final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); - resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", - key, - indent("\t\t", entry.getValue())); - - if (printResults) - resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); - - return false; - } - } - return true; - } - - private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, - final Iterable> list) { - final StringBuilder stringBuilder = new StringBuilder(); - for (final ConsumerRecord record : list) { - stringBuilder.append(prefix).append(record).append('\n'); - } - return stringBuilder.toString(); - } - - private static Long getSum(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) * (max - min + 1L) / 2L; - } - - private static Double getAvg(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) / 2.0; - } - - - private static boolean verifyTAgg(final PrintStream resultStream, - final Map> allData, - final Map>> taggEvents, - final boolean printResults) { - if (taggEvents == null) { - resultStream.println("tagg is missing"); - return false; - } else if (taggEvents.isEmpty()) { - resultStream.println("tagg is empty"); - return false; - } else { - resultStream.println("verifying tagg"); - - // generate expected answer - final Map expected = new HashMap<>(); - for (final String key : allData.keySet()) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - final String cnt = Long.toString(max - min + 1L); - - expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); - } - - // check the result - for (final Map.Entry>> entry : taggEvents.entrySet()) { - final String key = entry.getKey(); - Long expectedCount = expected.remove(key); - if (expectedCount == null) { - expectedCount = 0L; - } - - if (entry.getValue().getLast().value().longValue() != expectedCount) { - resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); - - if (printResults) - resultStream.println("\t taggEvents: " + entry.getValue()); - return false; - } - } - - } - return true; - } - - private static Number getMin(final String key) { - return Integer.parseInt(key.split("-")[0]); - } - - private static Number getMax(final String key) { - return Integer.parseInt(key.split("-")[1]); - } - - private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { - final List partitions = new ArrayList<>(); - - for (final String topic : topics) { - for (final PartitionInfo info : consumer.partitionsFor(topic)) { - partitions.add(new TopicPartition(info.topic(), info.partition())); - } - } - return partitions; - } - -} diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java deleted file mode 100644 index e8ec04c23ca52..0000000000000 --- a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -import java.time.Instant; - -public class SmokeTestUtil { - - final static int END = Integer.MAX_VALUE; - - static ProcessorSupplier printProcessorSupplier(final String topic) { - return printProcessorSupplier(topic, ""); - } - - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return new ProcessorSupplier() { - @Override - public Processor get() { - return new AbstractProcessor() { - private int numRecordsProcessed = 0; - private long smallestOffset = Long.MAX_VALUE; - private long largestOffset = Long.MIN_VALUE; - - @Override - public void init(final ProcessorContext context) { - super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); - System.out.flush(); - numRecordsProcessed = 0; - smallestOffset = Long.MAX_VALUE; - largestOffset = Long.MIN_VALUE; - } - - @Override - public void process(final Object key, final Object value) { - numRecordsProcessed++; - if (numRecordsProcessed % 100 == 0) { - System.out.printf("%s: %s%n", name, Instant.now()); - System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); - } - - if (smallestOffset > context().offset()) { - smallestOffset = context().offset(); - } - if (largestOffset < context().offset()) { - largestOffset = context().offset(); - } - } - - @Override - public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); - System.out.println("processed " + numRecordsProcessed + " records"); - final long processed; - if (largestOffset >= smallestOffset) { - processed = 1L + largestOffset - smallestOffset; - } else { - processed = 0L; - } - System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); - System.out.flush(); - } - }; - } - }; - } - - public static final class Unwindow implements KeyValueMapper, V, K> { - @Override - public K apply(final Windowed winKey, final V value) { - return winKey.key(); - } - } - - public static class Agg { - - KeyValueMapper> selector() { - return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); - } - - public Initializer init() { - return () -> 0L; - } - - Aggregator adder() { - return (aggKey, value, aggregate) -> aggregate + value; - } - - Aggregator remover() { - return (aggKey, value, aggregate) -> aggregate - value; - } - } - - public static Serde stringSerde = Serdes.String(); - - public static Serde intSerde = Serdes.Integer(); - - static Serde longSerde = Serdes.Long(); - - static Serde doubleSerde = Serdes.Double(); - - public static void sleep(final long duration) { - try { - Thread.sleep(duration); - } catch (final Exception ignore) { } - } - -} diff --git a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java deleted file mode 100644 index 07c7d5d82ebff..0000000000000 --- a/streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.StreamsConfig; - -import java.io.IOException; -import java.time.Duration; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; - -import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; -import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; - -public class StreamsSmokeTest { - - /** - * args ::= kafka propFileName command disableAutoTerminate - * command := "run" | "process" - * - * @param args - */ - public static void main(final String[] args) throws IOException { - if (args.length < 2) { - System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); - System.exit(1); - } - - final String propFileName = args[0]; - final String command = args[1]; - final boolean disableAutoTerminate = args.length > 2; - - final Properties streamsProperties = Utils.loadProps(propFileName); - final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); - - if (kafka == null) { - System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - System.exit(1); - } - - if ("process".equals(command)) { - if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { - - System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + - StreamsConfig.EXACTLY_ONCE); - - System.exit(1); - } - } - - System.out.println("StreamsTest instance started (StreamsSmokeTest)"); - System.out.println("command=" + command); - System.out.println("props=" + streamsProperties); - System.out.println("disableAutoTerminate=" + disableAutoTerminate); - - switch (command) { - case "run": - // this starts the driver (data generation and result verification) - final int numKeys = 10; - final int maxRecordsPerKey = 500; - if (disableAutoTerminate) { - generatePerpetually(kafka, numKeys, maxRecordsPerKey); - } else { - // slow down data production to span 30 seconds so that system tests have time to - // do their bounces, etc. - final Map> allData = - generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); - SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); - } - break; - case "process": - // this starts the stream processing app - new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); - break; - default: - System.out.println("unknown command: " + command); - } - } - -} diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 862bd34350304..4aa21d410043c 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -37,7 +37,7 @@ # can be replaced with metadata_2_versions backward_compatible_metadata_2_versions = [str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)] metadata_3_or_higher_versions = [str(LATEST_2_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(DEV_VERSION)] -smoke_test_versions = [str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(DEV_VERSION)] +smoke_test_versions = [str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5)] dev_version = [str(DEV_VERSION)] @@ -299,7 +299,8 @@ def test_app_upgrade(self, from_version, to_version): # shutdown self.driver.stop() - # ideally, we would actually verify the expected results, but it + # Ideally, we would actually verify the expected results. + # See KAFKA-10202 random.shuffle(self.processors) for p in self.processors: From 2465162c456560549b7678e3b13ed505023f8737 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Thu, 25 Jun 2020 22:24:27 -0500 Subject: [PATCH 18/23] reduce cyclomatic complexity --- .../InMemoryTimeOrderedKeyValueBuffer.java | 78 ++---------- ...eBufferChangelogDeserializationHelper.java | 111 ++++++++++++++++++ 2 files changed, 118 insertions(+), 71 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 89e8d049c9173..12c5c83455eb7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -38,6 +38,7 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.DeserializationResult; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; import java.nio.ByteBuffer; @@ -55,6 +56,8 @@ import java.util.function.Supplier; import static java.util.Objects.requireNonNull; +import static org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.deserializeV3; +import static org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.duckTypeV2; public final class InMemoryTimeOrderedKeyValueBuffer implements TimeOrderedKeyValueBuffer { private static final BytesSerializer KEY_SERIALIZER = new BytesSerializer(); @@ -363,37 +366,11 @@ private void restoreBatch(final Collection> batch ) ); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_2_CHANGELOG_HEADER_VALUE)) { - DeserializationResult deserializationResult = null; - RuntimeException v2DeserializationException = null; - RuntimeException v3DeserializationException = null; - try { - deserializationResult = getDeserializationResultV2(record, key); - } catch (final RuntimeException e) { - v2DeserializationException = e; - } - // versions 2.4.0, 2.4.1, and 2.5.0 would have erroneously encoded a V3 record with the - // V2 header, so we'll try duck-typing to see if this is decodable as V3 - if (deserializationResult == null) { - try { - deserializationResult = getDeserializationResultV3(record, key); - } catch (final RuntimeException e) { - v3DeserializationException = e; - } - } - - if (deserializationResult == null) { - // ok, it wasn't V3 either. Throw both exceptions: - final RuntimeException exception = - new RuntimeException("Couldn't deserialize record as v2 or v3: " + record, - v2DeserializationException); - exception.addSuppressed(v3DeserializationException); - throw exception; - } else { - cleanPut(deserializationResult.time, deserializationResult.key, deserializationResult.bufferValue); - } + final DeserializationResult deserializationResult = duckTypeV2(record, key); + cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_3_CHANGELOG_HEADER_VALUE)) { - final DeserializationResult deserializationResult = getDeserializationResultV3(record, key); - cleanPut(deserializationResult.time, deserializationResult.key, deserializationResult.bufferValue); + final DeserializationResult deserializationResult = deserializeV3(record, key); + cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); } else { throw new IllegalArgumentException("Restoring apparently invalid changelog record: " + record); } @@ -402,47 +379,6 @@ private void restoreBatch(final Collection> batch updateBufferMetrics(); } - private static InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV3(final ConsumerRecord record, - final Bytes key) { - final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); - final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); - final long time = valueAndTime.getLong(); - return new DeserializationResult(time, key, bufferValue); - } - - private static InMemoryTimeOrderedKeyValueBuffer.DeserializationResult getDeserializationResultV2(final ConsumerRecord record, - final Bytes key) { - final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); - final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); - final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); - - final int priorValueLength = valueAndTime.getInt(); - final byte[] priorValue; - if (priorValueLength == -1) { - priorValue = null; - } else { - priorValue = new byte[priorValueLength]; - valueAndTime.get(priorValue); - } - final long time = valueAndTime.getLong(); - final BufferValue bufferValue = new BufferValue(priorValue, change.oldValue, change.newValue, contextualRecord.recordContext()); - return new DeserializationResult(time, key, bufferValue); - } - - private static final class DeserializationResult { - - private final long time; - private final Bytes key; - private final BufferValue bufferValue; - - public DeserializationResult(final long time, final Bytes key, final BufferValue bufferValue) { - - this.time = time; - this.key = key; - this.bufferValue = bufferValue; - } - } - @Override public void evictWhile(final Supplier predicate, final Consumer> callback) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java new file mode 100644 index 0000000000000..2788adf1f78a5 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.kstream.internals.FullChangeSerde; + +import java.nio.ByteBuffer; + +import static java.util.Objects.requireNonNull; + +final class TimeOrderedKeyValueBufferChangelogDeserializationHelper { + private TimeOrderedKeyValueBufferChangelogDeserializationHelper() {} + + static final class DeserializationResult { + private final long time; + private final Bytes key; + private final BufferValue bufferValue; + + private DeserializationResult(final long time, final Bytes key, final BufferValue bufferValue) { + this.time = time; + this.key = key; + this.bufferValue = bufferValue; + } + + long time() { + return time; + } + + Bytes key() { + return key; + } + + BufferValue bufferValue() { + return bufferValue; + } + } + + + static DeserializationResult duckTypeV2(final ConsumerRecord record, final Bytes key) { + DeserializationResult deserializationResult = null; + RuntimeException v2DeserializationException = null; + RuntimeException v3DeserializationException = null; + try { + deserializationResult = deserializeV2(record, key); + } catch (final RuntimeException e) { + v2DeserializationException = e; + } + // versions 2.4.0, 2.4.1, and 2.5.0 would have erroneously encoded a V3 record with the + // V2 header, so we'll try duck-typing to see if this is decodable as V3 + if (deserializationResult == null) { + try { + deserializationResult = deserializeV3(record, key); + } catch (final RuntimeException e) { + v3DeserializationException = e; + } + } + + if (deserializationResult == null) { + // ok, it wasn't V3 either. Throw both exceptions: + final RuntimeException exception = + new RuntimeException("Couldn't deserialize record as v2 or v3: " + record, + v2DeserializationException); + exception.addSuppressed(v3DeserializationException); + throw exception; + } + return deserializationResult; + } + + static DeserializationResult deserializeV2(final ConsumerRecord record, + final Bytes key) { + final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); + final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); + final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); + + final int priorValueLength = valueAndTime.getInt(); + final byte[] priorValue; + if (priorValueLength == -1) { + priorValue = null; + } else { + priorValue = new byte[priorValueLength]; + valueAndTime.get(priorValue); + } + final long time = valueAndTime.getLong(); + final BufferValue bufferValue = new BufferValue(priorValue, change.oldValue, change.newValue, contextualRecord.recordContext()); + return new DeserializationResult(time, key, bufferValue); + } + + static DeserializationResult deserializeV3(final ConsumerRecord record, final Bytes key) { + final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); + final BufferValue bufferValue = BufferValue.deserialize(valueAndTime); + final long time = valueAndTime.getLong(); + return new DeserializationResult(time, key, bufferValue); + } +} \ No newline at end of file From d513fe881aede6956c19dddae5827f9cc88a870e Mon Sep 17 00:00:00 2001 From: John Roesler Date: Fri, 26 Jun 2020 10:23:26 -0500 Subject: [PATCH 19/23] Remove ParallelGC jvm param I was getting this exception, and somehow, the parallel GC parameter was the culprit java.lang.OutOfMemoryError: Java heap space at org.apache.kafka.streams.kstream.internals.FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(FullChangeSerde.java:82) at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.deserializeV2(TimeOrderedKeyValueBufferChangelogDeserializationHelper.java:90) at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.duckTypeV2(TimeOrderedKeyValueBufferChangelogDeserializationHelper.java:61) at org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer.restoreBatch(InMemoryTimeOrderedKeyValueBuffer.java:369) at org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer$$Lambda$284/0x00000001002cb440.restoreBatch(Unknown Source) at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferTest.shouldRestoreV3FormatWithV2Header(TimeOrderedKeyValueBufferTest.java:742) --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 180e1fc4e7242..6494c42932718 100644 --- a/build.gradle +++ b/build.gradle @@ -97,7 +97,7 @@ ext { buildVersionFileName = "kafka-version.properties" defaultMaxHeapSize = "2g" - defaultJvmArgs = ["-Xss4m", "-XX:+UseParallelGC"] + defaultJvmArgs = ["-Xss4m"] userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null From 7b8deb8d98fbf435886cbbcc9093016ee8dd2004 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Fri, 26 Jun 2020 10:44:38 -0500 Subject: [PATCH 20/23] cleanup --- bin/kafka-run-class.sh | 1 - .../InMemoryTimeOrderedKeyValueBuffer.java | 15 ++- ...eBufferChangelogDeserializationHelper.java | 4 +- .../TimeOrderedKeyValueBufferTest.java | 3 +- .../kafka/streams/tests/SmokeTestClient.java | 6 - .../kafka/streams/tests/SmokeTestDriver.java | 20 ++-- .../tests/streams/streams_upgrade_test.py | 110 +++++++++--------- 7 files changed, 78 insertions(+), 81 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 92d13413f6b92..95f1a3b606633 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -311,7 +311,6 @@ CLASSPATH=${CLASSPATH#:} # If Cygwin is detected, classpath is converted to Windows format. (( CYGWIN )) && CLASSPATH=$(cygpath --path --mixed "${CLASSPATH}") -echo "CLASSPATH=${CLASSPATH}" # Launch mode if [ "x$DAEMON_MODE" = "xtrue" ]; then nohup "$JAVA" $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp "$CLASSPATH" $KAFKA_OPTS "$@" > "$CONSOLE_OUTPUT_FILE" 2>&1 < /dev/null & diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 12c5c83455eb7..0390777ff6ca3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -65,7 +65,7 @@ public final class InMemoryTimeOrderedKeyValueBuffer implements TimeOrdere private static final byte[] V_1_CHANGELOG_HEADER_VALUE = {(byte) 1}; private static final byte[] V_2_CHANGELOG_HEADER_VALUE = {(byte) 2}; private static final byte[] V_3_CHANGELOG_HEADER_VALUE = {(byte) 3}; - private static final RecordHeaders CHANGELOG_HEADERS = + static final RecordHeaders CHANGELOG_HEADERS = new RecordHeaders(new Header[] {new RecordHeader("v", V_3_CHANGELOG_HEADER_VALUE)}); private static final String METRIC_SCOPE = "in-memory-suppression"; @@ -314,7 +314,8 @@ private void restoreBatch(final Collection> batch } } } else { - if (record.headers().lastHeader("v") == null) { + final Header versionHeader = record.headers().lastHeader("v"); + if (versionHeader == null) { // in this case, the changelog value is just the serialized record value final ByteBuffer timeAndValue = ByteBuffer.wrap(record.value()); final long time = timeAndValue.getLong(); @@ -343,7 +344,7 @@ private void restoreBatch(final Collection> batch recordContext ) ); - } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_1_CHANGELOG_HEADER_VALUE)) { + } else if (Arrays.equals(versionHeader.value(), V_1_CHANGELOG_HEADER_VALUE)) { // in this case, the changelog value is a serialized ContextualRecord final ByteBuffer timeAndValue = ByteBuffer.wrap(record.value()); final long time = timeAndValue.getLong(); @@ -365,12 +366,16 @@ private void restoreBatch(final Collection> batch contextualRecord.recordContext() ) ); - } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_2_CHANGELOG_HEADER_VALUE)) { + } else if (Arrays.equals(versionHeader.value(), V_2_CHANGELOG_HEADER_VALUE)) { + final DeserializationResult deserializationResult = duckTypeV2(record, key); cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); - } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_3_CHANGELOG_HEADER_VALUE)) { + + } else if (Arrays.equals(versionHeader.value(), V_3_CHANGELOG_HEADER_VALUE)) { + final DeserializationResult deserializationResult = deserializeV3(record, key); cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); + } else { throw new IllegalArgumentException("Restoring apparently invalid changelog record: " + record); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java index 2788adf1f78a5..14ec7a01ce69d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java @@ -83,8 +83,8 @@ static DeserializationResult duckTypeV2(final ConsumerRecord rec return deserializationResult; } - static DeserializationResult deserializeV2(final ConsumerRecord record, - final Bytes key) { + private static DeserializationResult deserializeV2(final ConsumerRecord record, + final Bytes key) { final ByteBuffer valueAndTime = ByteBuffer.wrap(record.value()); final ContextualRecord contextualRecord = ContextualRecord.deserialize(valueAndTime); final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 8c4904a6fd680..a054ac9390cdc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -55,14 +55,13 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Arrays.asList; import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer.CHANGELOG_HEADERS; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; import static org.junit.Assert.fail; @RunWith(Parameterized.class) public class TimeOrderedKeyValueBufferTest> { - private static final RecordHeaders CHANGELOG_HEADERS = - new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 3})}); private static final String APP_ID = "test-app"; private final Function bufferSupplier; diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index c518491f5b54f..c9d7bc749f810 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -55,7 +55,6 @@ public class SmokeTestClient extends SmokeTestUtil { private KafkaStreams streams; private boolean uncaughtException = false; - private boolean started; private volatile boolean closed; private static void addShutdownHook(final String name, final Runnable runnable) { @@ -92,10 +91,6 @@ public SmokeTestClient(final String name) { this.name = name; } - public boolean started() { - return started; - } - public boolean closed() { return closed; } @@ -108,7 +103,6 @@ public void start(final Properties streamsProperties) { streams.setStateListener((newState, oldState) -> { System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; countDownLatch.countDown(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java index 6fd70e06c0dc0..ac83cd95ebaaf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -487,14 +487,14 @@ private static boolean verify(final PrintStream resultStream, if (printResults) { resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); @@ -525,8 +525,8 @@ private static boolean verifySuppressed(final PrintStream resultStream, if (printResults) resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); return false; } diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 4aa21d410043c..37f177a82385f 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -192,25 +192,41 @@ def test_upgrade_downgrade_brokers(self, from_version, to_version): processor.stop() processor.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % processor.STDOUT_FILE, allow_fail=False) - @matrix(from_version=metadata_1_versions, to_version=backward_compatible_metadata_2_versions) - @matrix(from_version=metadata_1_versions, to_version=metadata_3_or_higher_versions) - @matrix(from_version=metadata_2_versions, to_version=metadata_3_or_higher_versions) - def test_metadata_upgrade(self, from_version, to_version): + @matrix(from_version=smoke_test_versions, to_version=dev_version) + def test_app_upgrade(self, from_version, to_version): """ - Starts 3 KafkaStreams instances with version and upgrades one-by-one to + Starts 3 KafkaStreams instances with , and upgrades one-by-one to """ + if from_version == to_version: + return + self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() - self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics=self.topics) + self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics={ + 'echo' : { 'partitions': 5, 'replication-factor': 1 }, + 'data' : { 'partitions': 5, 'replication-factor': 1 }, + 'min' : { 'partitions': 5, 'replication-factor': 1 }, + 'min-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, + 'min-raw' : { 'partitions': 5, 'replication-factor': 1 }, + 'max' : { 'partitions': 5, 'replication-factor': 1 }, + 'sum' : { 'partitions': 5, 'replication-factor': 1 }, + 'sws-raw' : { 'partitions': 5, 'replication-factor': 1 }, + 'sws-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, + 'dif' : { 'partitions': 5, 'replication-factor': 1 }, + 'cnt' : { 'partitions': 5, 'replication-factor': 1 }, + 'avg' : { 'partitions': 5, 'replication-factor': 1 }, + 'wcnt' : { 'partitions': 5, 'replication-factor': 1 }, + 'tagg' : { 'partitions': 5, 'replication-factor': 1 } + }) self.kafka.start() self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.driver.disable_auto_terminate() - self.processor1 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) - self.processor2 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) - self.processor3 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) + self.processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) + self.processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) self.driver.start() self.start_all_nodes_with(from_version) @@ -220,66 +236,47 @@ def test_metadata_upgrade(self, from_version, to_version): counter = 1 random.seed() - # first rolling bounce + # upgrade one-by-one via rolling bounce random.shuffle(self.processors) for p in self.processors: p.CLEAN_NODE_ENABLED = False - self.do_stop_start_bounce(p, from_version[:-2], to_version, counter) - counter = counter + 1 - - # second rolling bounce - random.shuffle(self.processors) - for p in self.processors: self.do_stop_start_bounce(p, None, to_version, counter) counter = counter + 1 # shutdown self.driver.stop() + # Ideally, we would actually verify the expected results. + # See KAFKA-10202 + random.shuffle(self.processors) for p in self.processors: node = p.node with node.account.monitor_log(p.STDOUT_FILE) as monitor: p.stop() - monitor.wait_until("UPGRADE-TEST-CLIENT-CLOSED", + monitor.wait_until("SMOKE-TEST-CLIENT-CLOSED", timeout_sec=60, - err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) + err_msg="Never saw output 'SMOKE-TEST-CLIENT-CLOSED' on " + str(node.account)) - @matrix(from_version=smoke_test_versions, to_version=dev_version) - def test_app_upgrade(self, from_version, to_version): + @matrix(from_version=metadata_1_versions, to_version=backward_compatible_metadata_2_versions) + @matrix(from_version=metadata_1_versions, to_version=metadata_3_or_higher_versions) + @matrix(from_version=metadata_2_versions, to_version=metadata_3_or_higher_versions) + def test_metadata_upgrade(self, from_version, to_version): """ - Starts 3 KafkaStreams instances with , and upgrades one-by-one to + Starts 3 KafkaStreams instances with version and upgrades one-by-one to """ - if from_version == to_version: - return - self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() - self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics={ - 'echo' : { 'partitions': 5, 'replication-factor': 1 }, - 'data' : { 'partitions': 5, 'replication-factor': 1 }, - 'min' : { 'partitions': 5, 'replication-factor': 1 }, - 'min-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, - 'min-raw' : { 'partitions': 5, 'replication-factor': 1 }, - 'max' : { 'partitions': 5, 'replication-factor': 1 }, - 'sum' : { 'partitions': 5, 'replication-factor': 1 }, - 'sws-raw' : { 'partitions': 5, 'replication-factor': 1 }, - 'sws-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, - 'dif' : { 'partitions': 5, 'replication-factor': 1 }, - 'cnt' : { 'partitions': 5, 'replication-factor': 1 }, - 'avg' : { 'partitions': 5, 'replication-factor': 1 }, - 'wcnt' : { 'partitions': 5, 'replication-factor': 1 }, - 'tagg' : { 'partitions': 5, 'replication-factor': 1 } - }) + self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics=self.topics) self.kafka.start() self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.driver.disable_auto_terminate() - self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) - self.processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) - self.processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) + self.processor1 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor2 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor3 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) self.driver.start() self.start_all_nodes_with(from_version) @@ -289,27 +286,30 @@ def test_app_upgrade(self, from_version, to_version): counter = 1 random.seed() - # upgrade one-by-one via rolling bounce + # first rolling bounce random.shuffle(self.processors) for p in self.processors: p.CLEAN_NODE_ENABLED = False + self.do_stop_start_bounce(p, from_version[:-2], to_version, counter) + counter = counter + 1 + + # second rolling bounce + random.shuffle(self.processors) + for p in self.processors: self.do_stop_start_bounce(p, None, to_version, counter) counter = counter + 1 # shutdown self.driver.stop() - # Ideally, we would actually verify the expected results. - # See KAFKA-10202 - random.shuffle(self.processors) for p in self.processors: node = p.node with node.account.monitor_log(p.STDOUT_FILE) as monitor: p.stop() - monitor.wait_until("SMOKE-TEST-CLIENT-CLOSED", + monitor.wait_until("UPGRADE-TEST-CLIENT-CLOSED", timeout_sec=60, - err_msg="Never saw output 'SMOKE-TEST-CLIENT-CLOSED' on " + str(node.account)) + err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) def test_version_probing_upgrade(self): """ @@ -380,18 +380,18 @@ def start_all_nodes_with(self, version): # double-check the version self.wait_for_verification(self.processor1, kafka_version_str, self.processor1.LOG_FILE) - self.wait_for_verification(self.processor2, kafka_version_str, self.processor1.LOG_FILE) - self.wait_for_verification(self.processor3, kafka_version_str, self.processor1.LOG_FILE) + self.wait_for_verification(self.processor2, kafka_version_str, self.processor2.LOG_FILE) + self.wait_for_verification(self.processor3, kafka_version_str, self.processor3.LOG_FILE) # wait for the members to join self.wait_for_verification(self.processor1, "SMOKE-TEST-CLIENT-STARTED", self.processor1.STDOUT_FILE) - self.wait_for_verification(self.processor2, "SMOKE-TEST-CLIENT-STARTED", self.processor1.STDOUT_FILE) - self.wait_for_verification(self.processor3, "SMOKE-TEST-CLIENT-STARTED", self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor2, "SMOKE-TEST-CLIENT-STARTED", self.processor2.STDOUT_FILE) + self.wait_for_verification(self.processor3, "SMOKE-TEST-CLIENT-STARTED", self.processor3.STDOUT_FILE) # make sure they've processed something self.wait_for_verification(self.processor1, self.processed_msg, self.processor1.STDOUT_FILE) - self.wait_for_verification(self.processor2, self.processed_msg, self.processor1.STDOUT_FILE) - self.wait_for_verification(self.processor3, self.processed_msg, self.processor1.STDOUT_FILE) + self.wait_for_verification(self.processor2, self.processed_msg, self.processor2.STDOUT_FILE) + self.wait_for_verification(self.processor3, self.processed_msg, self.processor3.STDOUT_FILE) def wait_for_verification(self, processor, message, file, num_lines=1): wait_until(lambda: self.verify_from_file(processor, message, file) >= num_lines, From 844321f187bb9cd4641ce087b632181761af1e5f Mon Sep 17 00:00:00 2001 From: John Roesler Date: Fri, 26 Jun 2020 16:24:24 -0500 Subject: [PATCH 21/23] fix attempt to allocate arbitrary sized array --- build.gradle | 2 +- .../org/apache/kafka/common/utils/Utils.java | 32 ++++++ .../apache/kafka/common/utils/UtilsTest.java | 72 ++++++++++++-- .../kstream/internals/FullChangeSerde.java | 16 +-- .../internals/ProcessorRecordContext.java | 23 ++--- .../streams/state/internals/BufferValue.java | 25 ++--- .../state/internals/ContextualRecord.java | 32 +----- .../InMemoryTimeOrderedKeyValueBuffer.java | 98 ++++++++----------- ...eBufferChangelogDeserializationHelper.java | 65 ++++++++++-- 9 files changed, 218 insertions(+), 147 deletions(-) diff --git a/build.gradle b/build.gradle index 6494c42932718..180e1fc4e7242 100644 --- a/build.gradle +++ b/build.gradle @@ -97,7 +97,7 @@ ext { buildVersionFileName = "kafka-version.properties" defaultMaxHeapSize = "2g" - defaultJvmArgs = ["-Xss4m"] + defaultJvmArgs = ["-Xss4m", "-XX:+UseParallelGC"] userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index cfe4b04454149..d89d57ac9e355 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.utils; +import java.nio.BufferUnderflowException; import java.util.EnumSet; import java.util.SortedSet; import java.util.TreeSet; @@ -284,6 +285,37 @@ public static byte[] toArray(ByteBuffer buffer, int offset, int size) { return dest; } + /** + * Starting from the current position, read an integer indicating the size of the byte array to read, + * then read the array. Consumes the buffer: upon returning, the buffer's position is after the array + * that is returned. + * @param buffer The buffer to read a size-prefixed array from + * @return The array + */ + public static byte[] getNullableSizePrefixedArray(final ByteBuffer buffer) { + final int size = buffer.getInt(); + return getNullableArray(buffer, size); + } + + /** + * Read a byte array of the given size. Consumes the buffer: upon returning, the buffer's position + * is after the array that is returned. + * @param buffer The buffer to read a size-prefixed array from + * @param size The number of bytes to read out of the buffer + * @return The array + */ + public static byte[] getNullableArray(final ByteBuffer buffer, final int size) { + if (size > buffer.remaining()) { + // preemptively throw this when the read is doomed to fail, so we don't have to allocate the array. + throw new BufferUnderflowException(); + } + final byte[] oldBytes = size == -1 ? null : new byte[size]; + if (oldBytes != null) { + buffer.get(oldBytes); + } + return oldBytes; + } + /** * Returns a copy of src byte array * @param src The byte array to copy diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index e134f7681be09..47eefe0d9a883 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -26,6 +26,7 @@ import java.io.EOFException; import java.io.File; import java.io.IOException; +import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.nio.charset.StandardCharsets; @@ -63,6 +64,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -83,7 +85,7 @@ public void testMurmur2() { cases.put("a-little-bit-long-string".getBytes(), -985981536); cases.put("a-little-bit-longer-string".getBytes(), -1486304829); cases.put("lkjh234lh9fiuh90y23oiuhsafujhadof229phr9h19h89h8".getBytes(), -58897971); - cases.put(new byte[]{'a', 'b', 'c'}, 479470107); + cases.put(new byte[] {'a', 'b', 'c'}, 479470107); for (Map.Entry c : cases.entrySet()) { assertEquals(c.getValue().intValue(), murmur2(c.getKey())); @@ -216,6 +218,65 @@ public void toArrayDirectByteBuffer() { assertEquals(2, buffer.position()); } + @Test + public void getNullableSizePrefixedArrayExact() { + byte[] input = {0, 0, 0, 2, 1, 0}; + final ByteBuffer buffer = ByteBuffer.wrap(input); + final byte[] array = Utils.getNullableSizePrefixedArray(buffer); + assertArrayEquals(new byte[] {1, 0}, array); + assertEquals(6, buffer.position()); + assertFalse(buffer.hasRemaining()); + } + + @Test + public void getNullableSizePrefixedArrayExactEmpty() { + byte[] input = {0, 0, 0, 0}; + final ByteBuffer buffer = ByteBuffer.wrap(input); + final byte[] array = Utils.getNullableSizePrefixedArray(buffer); + assertArrayEquals(new byte[] {}, array); + assertEquals(4, buffer.position()); + assertFalse(buffer.hasRemaining()); + } + + @Test + public void getNullableSizePrefixedArrayRemainder() { + byte[] input = {0, 0, 0, 2, 1, 0, 9}; + final ByteBuffer buffer = ByteBuffer.wrap(input); + final byte[] array = Utils.getNullableSizePrefixedArray(buffer); + assertArrayEquals(new byte[] {1, 0}, array); + assertEquals(6, buffer.position()); + assertTrue(buffer.hasRemaining()); + } + + @Test + public void getNullableSizePrefixedArrayNull() { + // -1 + byte[] input = {-1, -1, -1, -1}; + final ByteBuffer buffer = ByteBuffer.wrap(input); + final byte[] array = Utils.getNullableSizePrefixedArray(buffer); + assertNull(array); + assertEquals(4, buffer.position()); + assertFalse(buffer.hasRemaining()); + } + + @Test + public void getNullableSizePrefixedArrayInvalid() { + // -2 + byte[] input = {-1, -1, -1, -2}; + final ByteBuffer buffer = ByteBuffer.wrap(input); + assertThrows(NegativeArraySizeException.class, () -> Utils.getNullableSizePrefixedArray(buffer)); + } + + @Test + public void getNullableSizePrefixedArrayUnderflow() { + // Integer.MAX_VALUE + byte[] input = {127, -1, -1, -1}; + final ByteBuffer buffer = ByteBuffer.wrap(input); + // note, we get a buffer underflow exception instead of an OOME, even though the encoded size + // would be 2,147,483,647 aka 2.1 GB, probably larger than the available heap + assertThrows(BufferUnderflowException.class, () -> Utils.getNullableSizePrefixedArray(buffer)); + } + @Test public void utf8ByteArraySerde() { String utf8String = "A\u00ea\u00f1\u00fcC"; @@ -427,7 +488,7 @@ public void testReadFullyOrFailWithPartialFileChannelReads() throws IOException String expectedBufferContent = fileChannelMockExpectReadWithRandomBytes(channelMock, bufferSize); Utils.readFullyOrFail(channelMock, buffer, 0L, "test"); assertEquals("The buffer should be populated correctly", expectedBufferContent, - new String(buffer.array())); + new String(buffer.array())); assertFalse("The buffer should be filled", buffer.hasRemaining()); verify(channelMock, atLeastOnce()).read(any(), anyLong()); } @@ -444,7 +505,7 @@ public void testReadFullyWithPartialFileChannelReads() throws IOException { ByteBuffer buffer = ByteBuffer.allocate(bufferSize); Utils.readFully(channelMock, buffer, 0L); assertEquals("The buffer should be populated correctly.", expectedBufferContent, - new String(buffer.array())); + new String(buffer.array())); assertFalse("The buffer should be filled", buffer.hasRemaining()); verify(channelMock, atLeastOnce()).read(any(), anyLong()); } @@ -493,7 +554,7 @@ public void testLoadProps() throws IOException { * * @param channelMock The mocked FileChannel object * @param bufferSize The buffer size - * @return Expected buffer string + * @return Expected buffer string * @throws IOException If an I/O error occurs */ private String fileChannelMockExpectReadWithRandomBytes(final FileChannel channelMock, @@ -530,8 +591,7 @@ private static class TestCloseable implements Closeable { @Override public void close() throws IOException { closed = true; - if (closeException != null) - throw closeException; + if (closeException != null) { throw closeException; } } static TestCloseable[] createCloseables(boolean... exceptionOnClose) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java index 5a4150a2887d7..3a3439448cbe0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FullChangeSerde.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import static java.util.Objects.requireNonNull; +import static org.apache.kafka.common.utils.Utils.getNullableSizePrefixedArray; public final class FullChangeSerde { private final Serde inner; @@ -77,19 +78,8 @@ public static Change decomposeLegacyFormattedArrayIntoChangeArrays(final return null; } final ByteBuffer buffer = ByteBuffer.wrap(data); - - final int oldSize = buffer.getInt(); - final byte[] oldBytes = oldSize == -1 ? null : new byte[oldSize]; - if (oldBytes != null) { - buffer.get(oldBytes); - } - - final int newSize = buffer.getInt(); - final byte[] newBytes = newSize == -1 ? null : new byte[newSize]; - if (newBytes != null) { - buffer.get(newBytes); - } - + final byte[] oldBytes = getNullableSizePrefixedArray(buffer); + final byte[] newBytes = getNullableSizePrefixedArray(buffer); return new Change<>(newBytes, oldBytes); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java index 5662417a21460..5dd00620196a6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java @@ -26,6 +26,8 @@ import java.util.Objects; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; +import static org.apache.kafka.common.utils.Utils.getNullableSizePrefixedArray; public class ProcessorRecordContext implements RecordContext { @@ -161,12 +163,10 @@ public byte[] serialize() { public static ProcessorRecordContext deserialize(final ByteBuffer buffer) { final long timestamp = buffer.getLong(); final long offset = buffer.getLong(); - final int topicSize = buffer.getInt(); final String topic; { - // not handling the null topic condition, because we believe the topic will never be null when we serialize - final byte[] topicBytes = new byte[topicSize]; - buffer.get(topicBytes); + // we believe the topic will never be null when we serialize + final byte[] topicBytes = requireNonNull(getNullableSizePrefixedArray(buffer)); topic = new String(topicBytes, UTF_8); } final int partition = buffer.getInt(); @@ -177,19 +177,8 @@ public static ProcessorRecordContext deserialize(final ByteBuffer buffer) { } else { final Header[] headerArr = new Header[headerCount]; for (int i = 0; i < headerCount; i++) { - final int keySize = buffer.getInt(); - final byte[] keyBytes = new byte[keySize]; - buffer.get(keyBytes); - - final int valueSize = buffer.getInt(); - final byte[] valueBytes; - if (valueSize == -1) { - valueBytes = null; - } else { - valueBytes = new byte[valueSize]; - buffer.get(valueBytes); - } - + final byte[] keyBytes = requireNonNull(getNullableSizePrefixedArray(buffer)); + final byte[] valueBytes = getNullableSizePrefixedArray(buffer); headerArr[i] = new RecordHeader(new String(keyBytes, UTF_8), valueBytes); } headers = new RecordHeaders(headerArr); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java index 225dbb5d803e1..f27ab19a002da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/BufferValue.java @@ -22,6 +22,9 @@ import java.util.Arrays; import java.util.Objects; +import static org.apache.kafka.common.utils.Utils.getNullableArray; +import static org.apache.kafka.common.utils.Utils.getNullableSizePrefixedArray; + public final class BufferValue { private static final int NULL_VALUE_SENTINEL = -1; private static final int OLD_PREV_DUPLICATE_VALUE_SENTINEL = -2; @@ -67,35 +70,21 @@ ProcessorRecordContext context() { static BufferValue deserialize(final ByteBuffer buffer) { final ProcessorRecordContext context = ProcessorRecordContext.deserialize(buffer); - final byte[] priorValue = extractValue(buffer); + final byte[] priorValue = getNullableSizePrefixedArray(buffer); final byte[] oldValue; final int oldValueLength = buffer.getInt(); - if (oldValueLength == NULL_VALUE_SENTINEL) { - oldValue = null; - } else if (oldValueLength == OLD_PREV_DUPLICATE_VALUE_SENTINEL) { + if (oldValueLength == OLD_PREV_DUPLICATE_VALUE_SENTINEL) { oldValue = priorValue; } else { - oldValue = new byte[oldValueLength]; - buffer.get(oldValue); + oldValue = getNullableArray(buffer, oldValueLength); } - final byte[] newValue = extractValue(buffer); + final byte[] newValue = getNullableSizePrefixedArray(buffer); return new BufferValue(priorValue, oldValue, newValue, context); } - private static byte[] extractValue(final ByteBuffer buffer) { - final int valueLength = buffer.getInt(); - if (valueLength == NULL_VALUE_SENTINEL) { - return null; - } else { - final byte[] value = new byte[valueLength]; - buffer.get(value); - return value; - } - } - ByteBuffer serialize(final int endPadding) { final int sizeOfValueLength = Integer.BYTES; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ContextualRecord.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ContextualRecord.java index 3c24f521c23ed..a26b4375c0cd7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ContextualRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ContextualRecord.java @@ -22,6 +22,8 @@ import java.util.Arrays; import java.util.Objects; +import static org.apache.kafka.common.utils.Utils.getNullableSizePrefixedArray; + public class ContextualRecord { private final byte[] value; private final ProcessorRecordContext recordContext; @@ -43,36 +45,10 @@ long residentMemorySizeEstimate() { return (value == null ? 0 : value.length) + recordContext.residentMemorySizeEstimate(); } - ByteBuffer serialize(final int endPadding) { - final byte[] serializedContext = recordContext.serialize(); - - final int sizeOfContext = serializedContext.length; - final int sizeOfValueLength = Integer.BYTES; - final int sizeOfValue = value == null ? 0 : value.length; - final ByteBuffer buffer = ByteBuffer.allocate(sizeOfContext + sizeOfValueLength + sizeOfValue + endPadding); - - buffer.put(serializedContext); - if (value == null) { - buffer.putInt(-1); - } else { - buffer.putInt(value.length); - buffer.put(value); - } - - return buffer; - } - static ContextualRecord deserialize(final ByteBuffer buffer) { final ProcessorRecordContext context = ProcessorRecordContext.deserialize(buffer); - - final int valueLength = buffer.getInt(); - if (valueLength == -1) { - return new ContextualRecord(null, context); - } else { - final byte[] value = new byte[valueLength]; - buffer.get(value); - return new ContextualRecord(value, context); - } + final byte[] value = getNullableSizePrefixedArray(buffer); + return new ContextualRecord(value, context); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 0390777ff6ca3..2909e2763f1a5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -56,6 +56,8 @@ import java.util.function.Supplier; import static java.util.Objects.requireNonNull; +import static org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.deserializeV0; +import static org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.deserializeV1; import static org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.deserializeV3; import static org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.duckTypeV2; @@ -316,66 +318,51 @@ private void restoreBatch(final Collection> batch } else { final Header versionHeader = record.headers().lastHeader("v"); if (versionHeader == null) { - // in this case, the changelog value is just the serialized record value - final ByteBuffer timeAndValue = ByteBuffer.wrap(record.value()); - final long time = timeAndValue.getLong(); - final byte[] changelogValue = new byte[record.value().length - 8]; - timeAndValue.get(changelogValue); - - final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(changelogValue)); - - final ProcessorRecordContext recordContext = new ProcessorRecordContext( - record.timestamp(), - record.offset(), - record.partition(), - record.topic(), - record.headers() - ); - - cleanPut( - time, - key, - new BufferValue( - index.containsKey(key) - ? internalPriorValueForBuffered(key) - : change.oldValue, - change.oldValue, - change.newValue, - recordContext - ) - ); - } else if (Arrays.equals(versionHeader.value(), V_1_CHANGELOG_HEADER_VALUE)) { - // in this case, the changelog value is a serialized ContextualRecord - final ByteBuffer timeAndValue = ByteBuffer.wrap(record.value()); - final long time = timeAndValue.getLong(); - final byte[] changelogValue = new byte[record.value().length - 8]; - timeAndValue.get(changelogValue); - - final ContextualRecord contextualRecord = ContextualRecord.deserialize(ByteBuffer.wrap(changelogValue)); - final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); - - cleanPut( - time, - key, - new BufferValue( - index.containsKey(key) - ? internalPriorValueForBuffered(key) - : change.oldValue, - change.oldValue, - change.newValue, - contextualRecord.recordContext() - ) - ); - } else if (Arrays.equals(versionHeader.value(), V_2_CHANGELOG_HEADER_VALUE)) { - - final DeserializationResult deserializationResult = duckTypeV2(record, key); + // Version 0: + // value: + // - buffer time + // - old value + // - new value + final byte[] previousBufferedValue = index.containsKey(key) + ? internalPriorValueForBuffered(key) + : null; + final DeserializationResult deserializationResult = deserializeV0(record, key, previousBufferedValue); cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); - } else if (Arrays.equals(versionHeader.value(), V_3_CHANGELOG_HEADER_VALUE)) { - + // Version 3: + // value: + // - record context + // - prior value + // - old value + // - new value + // - buffer time final DeserializationResult deserializationResult = deserializeV3(record, key); cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); + } else if (Arrays.equals(versionHeader.value(), V_2_CHANGELOG_HEADER_VALUE)) { + // Version 2: + // value: + // - record context + // - old value + // - new value + // - prior value + // - buffer time + // NOTE: 2.4.0, 2.4.1, and 2.5.0 actually encode Version 3 formatted data, + // but still set the Version 2 flag, so to deserialize, we have to duck type. + final DeserializationResult deserializationResult = duckTypeV2(record, key); + cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); + } else if (Arrays.equals(versionHeader.value(), V_1_CHANGELOG_HEADER_VALUE)) { + // Version 1: + // value: + // - buffer time + // - record context + // - old value + // - new value + final byte[] previousBufferedValue = index.containsKey(key) + ? internalPriorValueForBuffered(key) + : null; + final DeserializationResult deserializationResult = deserializeV1(record, key, previousBufferedValue); + cleanPut(deserializationResult.time(), deserializationResult.key(), deserializationResult.bufferValue()); } else { throw new IllegalArgumentException("Restoring apparently invalid changelog record: " + record); } @@ -384,6 +371,7 @@ private void restoreBatch(final Collection> batch updateBufferMetrics(); } + @Override public void evictWhile(final Supplier predicate, final Consumer> callback) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java index 14ec7a01ce69d..74489c230b99f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferChangelogDeserializationHelper.java @@ -18,8 +18,10 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.FullChangeSerde; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import java.nio.ByteBuffer; @@ -52,6 +54,59 @@ BufferValue bufferValue() { } } + static DeserializationResult deserializeV0(final ConsumerRecord record, + final Bytes key, + final byte[] previousBufferedValue) { + + final ByteBuffer timeAndValue = ByteBuffer.wrap(record.value()); + final long time = timeAndValue.getLong(); + final byte[] changelogValue = new byte[record.value().length - 8]; + timeAndValue.get(changelogValue); + + final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(changelogValue)); + + final ProcessorRecordContext recordContext = new ProcessorRecordContext( + record.timestamp(), + record.offset(), + record.partition(), + record.topic(), + record.headers() + ); + + return new DeserializationResult( + time, + key, + new BufferValue( + previousBufferedValue == null ? change.oldValue : previousBufferedValue, + change.oldValue, + change.newValue, + recordContext + ) + ); + } + + static DeserializationResult deserializeV1(final ConsumerRecord record, + final Bytes key, + final byte[] previousBufferedValue) { + final ByteBuffer timeAndValue = ByteBuffer.wrap(record.value()); + final long time = timeAndValue.getLong(); + final byte[] changelogValue = new byte[record.value().length - 8]; + timeAndValue.get(changelogValue); + + final ContextualRecord contextualRecord = ContextualRecord.deserialize(ByteBuffer.wrap(changelogValue)); + final Change change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); + + return new DeserializationResult( + time, + key, + new BufferValue( + previousBufferedValue == null ? change.oldValue : previousBufferedValue, + change.oldValue, + change.newValue, + contextualRecord.recordContext() + ) + ); + } static DeserializationResult duckTypeV2(final ConsumerRecord record, final Bytes key) { DeserializationResult deserializationResult = null; @@ -88,15 +143,7 @@ private static DeserializationResult deserializeV2(final ConsumerRecord change = requireNonNull(FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(contextualRecord.value())); - - final int priorValueLength = valueAndTime.getInt(); - final byte[] priorValue; - if (priorValueLength == -1) { - priorValue = null; - } else { - priorValue = new byte[priorValueLength]; - valueAndTime.get(priorValue); - } + final byte[] priorValue = Utils.getNullableSizePrefixedArray(valueAndTime); final long time = valueAndTime.getLong(); final BufferValue bufferValue = new BufferValue(priorValue, change.oldValue, change.newValue, contextualRecord.recordContext()); return new DeserializationResult(time, key, bufferValue); From c4200998e5361bc2500c66cfac4fd64045e20add Mon Sep 17 00:00:00 2001 From: John Roesler Date: Fri, 26 Jun 2020 16:59:20 -0500 Subject: [PATCH 22/23] factor out system test changes --- .../SmokeTestDriverIntegrationTest.java | 4 + .../kafka/streams/tests/SmokeTestClient.java | 130 ++-- .../kafka/streams/tests/SmokeTestUtil.java | 2 +- .../kafka/streams/tests/SmokeTestClient.java | 298 --------- .../kafka/streams/tests/SmokeTestDriver.java | 632 ------------------ .../kafka/streams/tests/SmokeTestUtil.java | 134 ---- .../kafka/streams/tests/StreamsSmokeTest.java | 99 --- .../kafka/streams/tests/SmokeTestClient.java | 298 --------- .../kafka/streams/tests/SmokeTestDriver.java | 622 ----------------- .../kafka/streams/tests/SmokeTestUtil.java | 134 ---- .../kafka/streams/tests/StreamsSmokeTest.java | 99 --- .../kafka/streams/tests/SmokeTestClient.java | 298 --------- .../kafka/streams/tests/SmokeTestDriver.java | 622 ----------------- .../kafka/streams/tests/SmokeTestUtil.java | 134 ---- .../kafka/streams/tests/StreamsSmokeTest.java | 99 --- .../kafka/streams/tests/SmokeTestClient.java | 298 --------- .../kafka/streams/tests/SmokeTestDriver.java | 622 ----------------- .../kafka/streams/tests/SmokeTestUtil.java | 134 ---- .../kafka/streams/tests/StreamsSmokeTest.java | 99 --- tests/kafkatest/services/streams.py | 50 +- .../tests/streams/streams_upgrade_test.py | 117 ++-- 21 files changed, 120 insertions(+), 4805 deletions(-) delete mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java delete mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java delete mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java delete mode 100644 streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java delete mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java delete mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java delete mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java delete mode 100644 streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java delete mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java delete mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java delete mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java delete mode 100644 streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java delete mode 100644 streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java index 0e4e78ff68f7f..c2bd7a5d24615 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java @@ -104,6 +104,10 @@ public void shouldWorkWithRebalance() throws InterruptedException { clients.add(smokeTestClient); smokeTestClient.start(props); + while (!clients.get(clients.size() - 1).started()) { + Thread.sleep(100); + } + // let the oldest client die of "natural causes" if (clients.size() >= 3) { final SmokeTestClient client = clients.remove(0); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index c9d7bc749f810..db243fdd849c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -16,10 +16,11 @@ */ package org.apache.kafka.streams.tests; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.utils.Exit; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -37,15 +38,11 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.test.TestUtils; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; import java.time.Duration; import java.time.Instant; import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; @@ -53,85 +50,37 @@ public class SmokeTestClient extends SmokeTestUtil { private final String name; + private Thread thread; private KafkaStreams streams; private boolean uncaughtException = false; - private volatile boolean closed; - - private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { - Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); - } else { - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } - } - - private static File tempDirectory() { - final String prefix = "kafka-"; - final File file; - try { - file = Files.createTempDirectory(prefix).toFile(); - } catch (final IOException ex) { - throw new RuntimeException("Failed to create a temp dir", ex); - } - file.deleteOnExit(); - - addShutdownHook("delete-temp-file-shutdown-hook", () -> { - try { - Utils.delete(file); - } catch (final IOException e) { - System.out.println("Error deleting " + file.getAbsolutePath()); - e.printStackTrace(System.out); - } - }); - - return file; - } + private boolean started; + private boolean closed; public SmokeTestClient(final String name) { + super(); this.name = name; } + public boolean started() { + return started; + } + public boolean closed() { return closed; } public void start(final Properties streamsProperties) { - final Topology build = getTopology(); - streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); - - final CountDownLatch countDownLatch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - countDownLatch.countDown(); - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - + streams = createKafkaStreams(streamsProperties); streams.setUncaughtExceptionHandler((t, e) -> { System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - e.printStackTrace(System.out); uncaughtException = true; - streams.close(Duration.ofSeconds(30)); + e.printStackTrace(); }); - addShutdownHook("streams-shutdown-hook", this::close); + Exit.addShutdownHook("streams-shutdown-hook", () -> close()); - streams.start(); - try { - if (!countDownLatch.await(1, TimeUnit.MINUTES)) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); - } - } catch (final InterruptedException e) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); - e.printStackTrace(System.out); - } - System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); - System.out.println(name + " started at " + Instant.now()); + thread = new Thread(() -> streams.start()); + thread.start(); } public void closeAsync() { @@ -139,14 +88,17 @@ public void closeAsync() { } public void close() { - final boolean wasClosed = streams.close(Duration.ofMinutes(1)); - - if (wasClosed && !uncaughtException) { + streams.close(Duration.ofSeconds(5)); + // do not remove these printouts since they are needed for health scripts + if (!uncaughtException) { System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } else if (wasClosed) { + } + try { + thread.join(); + } catch (final Exception ex) { + // do not remove these printouts since they are needed for health scripts System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - } else { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + // ignore } } @@ -154,11 +106,39 @@ private Properties getStreamsConfig(final Properties props) { final Properties fullProps = new Properties(props); fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); + fullProps.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); + fullProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); + fullProps.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + fullProps.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); + fullProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + fullProps.put(ProducerConfig.ACKS_CONFIG, "all"); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()); fullProps.putAll(props); return fullProps; } + private KafkaStreams createKafkaStreams(final Properties props) { + final Topology build = getTopology(); + final KafkaStreams streamsClient = new KafkaStreams(build, getStreamsConfig(props)); + streamsClient.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + streamsClient.setUncaughtExceptionHandler((t, e) -> { + System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + streamsClient.close(Duration.ofSeconds(30)); + }); + + return streamsClient; + } + public Topology getTopology() { final StreamsBuilder builder = new StreamsBuilder(); final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index e8ec04c23ca52..990bfdb73fede 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -75,7 +75,7 @@ public void process(final Object key, final Object value) { @Override public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.printf("Close processor for task %s", context().taskId()); System.out.println("processed " + numRecordsProcessed + " records"); final long processed; if (largestOffset >= smallestOffset) { diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java deleted file mode 100644 index ced1369af3855..0000000000000 --- a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ /dev/null @@ -1,298 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.Topology; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Grouped; -import org.apache.kafka.streams.kstream.KGroupedStream; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; -import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.streams.state.WindowStore; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; - -public class SmokeTestClient extends SmokeTestUtil { - - private final String name; - - private KafkaStreams streams; - private boolean uncaughtException = false; - private boolean started; - private volatile boolean closed; - - private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { - Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); - } else { - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } - } - - private static File tempDirectory() { - final String prefix = "kafka-"; - final File file; - try { - file = Files.createTempDirectory(prefix).toFile(); - } catch (final IOException ex) { - throw new RuntimeException("Failed to create a temp dir", ex); - } - file.deleteOnExit(); - - addShutdownHook("delete-temp-file-shutdown-hook", () -> { - try { - Utils.delete(file); - } catch (final IOException e) { - System.out.println("Error deleting " + file.getAbsolutePath()); - e.printStackTrace(System.out); - } - }); - - return file; - } - - public SmokeTestClient(final String name) { - this.name = name; - } - - public boolean started() { - return started; - } - - public boolean closed() { - return closed; - } - - public void start(final Properties streamsProperties) { - final Topology build = getTopology(); - streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); - - final CountDownLatch countDownLatch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; - countDownLatch.countDown(); - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - - streams.setUncaughtExceptionHandler((t, e) -> { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - e.printStackTrace(System.out); - uncaughtException = true; - streams.close(Duration.ofSeconds(30)); - }); - - addShutdownHook("streams-shutdown-hook", this::close); - - streams.start(); - try { - if (!countDownLatch.await(1, TimeUnit.MINUTES)) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); - } - } catch (final InterruptedException e) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); - e.printStackTrace(System.out); - } - System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); - System.out.println(name + " started at " + Instant.now()); - } - - public void closeAsync() { - streams.close(Duration.ZERO); - } - - public void close() { - final boolean closed = streams.close(Duration.ofMinutes(1)); - - if (closed && !uncaughtException) { - System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } else if (closed) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - } else { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); - } - } - - private Properties getStreamsConfig(final Properties props) { - final Properties fullProps = new Properties(props); - fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); - fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); - fullProps.putAll(props); - return fullProps; - } - - public Topology getTopology() { - final StreamsBuilder builder = new StreamsBuilder(); - final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); - final KStream source = builder.stream("data", stringIntConsumed); - source.filterNot((k, v) -> k.equals("flush")) - .to("echo", Produced.with(stringSerde, intSerde)); - final KStream data = source.filter((key, value) -> value == null || value != END); - data.process(SmokeTestUtil.printProcessorSupplier("data", name)); - - // min - final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); - - final KTable, Integer> minAggregation = groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) - .aggregate( - () -> Integer.MAX_VALUE, - (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, - Materialized - .>as("uwin-min") - .withValueSerde(intSerde) - .withRetention(Duration.ofHours(25)) - ); - - streamify(minAggregation, "min-raw"); - - streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); - - minAggregation - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("min", Produced.with(stringSerde, intSerde)); - - final KTable, Integer> smallWindowSum = groupedData - .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) - .reduce((l, r) -> l + r); - - streamify(smallWindowSum, "sws-raw"); - streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); - - final KTable minTable = builder.table( - "min", - Consumed.with(stringSerde, intSerde), - Materialized.as("minStoreName")); - - minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); - - // max - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> Integer.MIN_VALUE, - (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, - Materialized.>as("uwin-max").withValueSerde(intSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("max", Produced.with(stringSerde, intSerde)); - - final KTable maxTable = builder.table( - "max", - Consumed.with(stringSerde, intSerde), - Materialized.as("maxStoreName")); - maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); - - // sum - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> 0L, - (aggKey, value, aggregate) -> (long) value + aggregate, - Materialized.>as("win-sum").withValueSerde(longSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("sum", Produced.with(stringSerde, longSerde)); - - final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); - final KTable sumTable = builder.table("sum", stringLongConsumed); - sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); - - // cnt - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .count(Materialized.as("uwin-cnt")) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("cnt", Produced.with(stringSerde, longSerde)); - - final KTable cntTable = builder.table( - "cnt", - Consumed.with(stringSerde, longSerde), - Materialized.as("cntStoreName")); - cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); - - // dif - maxTable - .join( - minTable, - (value1, value2) -> value1 - value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("dif", Produced.with(stringSerde, intSerde)); - - // avg - sumTable - .join( - cntTable, - (value1, value2) -> (double) value1 / (double) value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("avg", Produced.with(stringSerde, doubleSerde)); - - // test repartition - final Agg agg = new Agg(); - cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) - .aggregate(agg.init(), agg.adder(), agg.remover(), - Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) - .withKeySerde(Serdes.String()) - .withValueSerde(Serdes.Long())) - .toStream() - .to("tagg", Produced.with(stringSerde, longSerde)); - - return builder.build(); - } - - private static void streamify(final KTable, Integer> windowedTable, final String topic) { - windowedTable - .toStream() - .filterNot((k, v) -> k.key().equals("flush")) - .map((key, value) -> new KeyValue<>(key.toString(), value)) - .to(topic, Produced.with(stringSerde, intSerde)); - } -} diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java deleted file mode 100644 index d0a7d22d938f8..0000000000000 --- a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ /dev/null @@ -1,632 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.utils.Utils; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; - -public class SmokeTestDriver extends SmokeTestUtil { - private static final String[] TOPICS = { - "data", - "echo", - "max", - "min", "min-suppressed", "min-raw", - "dif", - "sum", - "sws-raw", "sws-suppressed", - "cnt", - "avg", - "tagg" - }; - - private static final int MAX_RECORD_EMPTY_RETRIES = 30; - - private static class ValueList { - public final String key; - private final int[] values; - private int index; - - ValueList(final int min, final int max) { - key = min + "-" + max; - - values = new int[max - min + 1]; - for (int i = 0; i < values.length; i++) { - values[i] = min + i; - } - // We want to randomize the order of data to test not completely predictable processing order - // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) - // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window - shuffle(values, 10); - - index = 0; - } - - int next() { - return (index < values.length) ? values[index++] : -1; - } - } - - public static String[] topics() { - return Arrays.copyOf(TOPICS, TOPICS.length); - } - - static void generatePerpetually(final String kafka, - final int numKeys, - final int maxRecordsPerKey) { - final Properties producerProps = generatorProperties(kafka); - - int numRecordsProduced = 0; - - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - } - - final Random rand = new Random(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (true) { - final int index = rand.nextInt(numKeys); - final String key = data[index].key; - final int value = data[index].next(); - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record); - - numRecordsProduced++; - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(2); - } - } - } - - public static Map> generate(final String kafka, - final int numKeys, - final int maxRecordsPerKey, - final Duration timeToSpend) { - final Properties producerProps = generatorProperties(kafka); - - - int numRecordsProduced = 0; - - final Map> allData = new HashMap<>(); - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - allData.put(data[i].key, new HashSet<>()); - } - final Random rand = new Random(); - - int remaining = data.length; - - final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; - - List> needRetry = new ArrayList<>(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (remaining > 0) { - final int index = rand.nextInt(remaining); - final String key = data[index].key; - final int value = data[index].next(); - - if (value < 0) { - remaining--; - data[index] = data[remaining]; - } else { - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record, new TestCallback(record, needRetry)); - - numRecordsProduced++; - allData.get(key).add(value); - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(Math.max(recordPauseTime, 2)); - } - } - producer.flush(); - - int remainingRetries = 5; - while (!needRetry.isEmpty()) { - final List> needRetry2 = new ArrayList<>(); - for (final ProducerRecord record : needRetry) { - System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); - producer.send(record, new TestCallback(record, needRetry2)); - } - producer.flush(); - needRetry = needRetry2; - - if (--remainingRetries == 0 && !needRetry.isEmpty()) { - System.err.println("Failed to produce all records after multiple retries"); - Exit.exit(1); - } - } - - // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out - // all suppressed records. - final List partitions = producer.partitionsFor("data"); - for (final PartitionInfo partition : partitions) { - producer.send(new ProducerRecord<>( - partition.topic(), - partition.partition(), - System.currentTimeMillis() + Duration.ofDays(2).toMillis(), - stringSerde.serializer().serialize("", "flush"), - intSerde.serializer().serialize("", 0) - )); - } - } - return Collections.unmodifiableMap(allData); - } - - private static Properties generatorProperties(final String kafka) { - final Properties producerProps = new Properties(); - producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); - return producerProps; - } - - private static class TestCallback implements Callback { - private final ProducerRecord originalRecord; - private final List> needRetry; - - TestCallback(final ProducerRecord originalRecord, - final List> needRetry) { - this.originalRecord = originalRecord; - this.needRetry = needRetry; - } - - @Override - public void onCompletion(final RecordMetadata metadata, final Exception exception) { - if (exception != null) { - if (exception instanceof TimeoutException) { - needRetry.add(originalRecord); - } else { - exception.printStackTrace(); - Exit.exit(1); - } - } - } - } - - private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { - final Random rand = new Random(); - for (int i = 0; i < data.length; i++) { - // we shuffle data within windowSize - final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; - - // swap - final int tmp = data[i]; - data[i] = data[j]; - data[j] = tmp; - } - } - - public static class NumberDeserializer implements Deserializer { - @Override - public void configure(final Map configs, final boolean isKey) { - - } - - @Override - public Number deserialize(final String topic, final byte[] data) { - final Number value; - switch (topic) { - case "data": - case "echo": - case "min": - case "min-raw": - case "min-suppressed": - case "sws-raw": - case "sws-suppressed": - case "max": - case "dif": - value = intSerde.deserializer().deserialize(topic, data); - break; - case "sum": - case "cnt": - case "tagg": - value = longSerde.deserializer().deserialize(topic, data); - break; - case "avg": - value = doubleSerde.deserializer().deserialize(topic, data); - break; - default: - throw new RuntimeException("unknown topic: " + topic); - } - return value; - } - - @Override - public void close() { - - } - } - - public static VerificationResult verify(final String kafka, - final Map> inputs, - final int maxRecordsPerKey) { - final Properties props = new Properties(); - props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); - props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); - - final KafkaConsumer consumer = new KafkaConsumer<>(props); - final List partitions = getAllPartitions(consumer, TOPICS); - consumer.assign(partitions); - consumer.seekToBeginning(partitions); - - final int recordsGenerated = inputs.size() * maxRecordsPerKey; - int recordsProcessed = 0; - final Map processed = - Stream.of(TOPICS) - .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); - - final Map>>> events = new HashMap<>(); - - VerificationResult verificationResult = new VerificationResult(false, "no results yet"); - int retry = 0; - final long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { - final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (records.isEmpty() && recordsProcessed >= recordsGenerated) { - verificationResult = verifyAll(inputs, events, false); - if (verificationResult.passed()) { - break; - } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { - System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); - break; - } else { - System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); - } - } else { - System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); - - retry = 0; - for (final ConsumerRecord record : records) { - final String key = record.key(); - - final String topic = record.topic(); - processed.get(topic).incrementAndGet(); - - if (topic.equals("echo")) { - recordsProcessed++; - if (recordsProcessed % 100 == 0) { - System.out.println("Echo records processed = " + recordsProcessed); - } - } - - events.computeIfAbsent(topic, t -> new HashMap<>()) - .computeIfAbsent(key, k -> new LinkedList<>()) - .add(record); - } - - System.out.println(processed); - } - } - consumer.close(); - final long finished = System.currentTimeMillis() - start; - System.out.println("Verification time=" + finished); - System.out.println("-------------------"); - System.out.println("Result Verification"); - System.out.println("-------------------"); - System.out.println("recordGenerated=" + recordsGenerated); - System.out.println("recordProcessed=" + recordsProcessed); - - if (recordsProcessed > recordsGenerated) { - System.out.println("PROCESSED-MORE-THAN-GENERATED"); - } else if (recordsProcessed < recordsGenerated) { - System.out.println("PROCESSED-LESS-THAN-GENERATED"); - } - - boolean success; - - final Map> received = - events.get("echo") - .entrySet() - .stream() - .map(entry -> mkEntry( - entry.getKey(), - entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) - ) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - success = inputs.equals(received); - - if (success) { - System.out.println("ALL-RECORDS-DELIVERED"); - } else { - int missedCount = 0; - for (final Map.Entry> entry : inputs.entrySet()) { - missedCount += received.get(entry.getKey()).size(); - } - System.out.println("missedRecords=" + missedCount); - } - - // give it one more try if it's not already passing. - if (!verificationResult.passed()) { - verificationResult = verifyAll(inputs, events, true); - } - success &= verificationResult.passed(); - - System.out.println(verificationResult.result()); - - System.out.println(success ? "SUCCESS" : "FAILURE"); - return verificationResult; - } - - public static class VerificationResult { - private final boolean passed; - private final String result; - - VerificationResult(final boolean passed, final String result) { - this.passed = passed; - this.result = result; - } - - public boolean passed() { - return passed; - } - - public String result() { - return result; - } - } - - private static VerificationResult verifyAll(final Map> inputs, - final Map>>> events, - final boolean printResults) { - final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - boolean pass; - try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { - pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); - pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); - pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { - final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); - return getMin(unwindowedKey); - }, printResults); - pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); - pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); - pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); - pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); - pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); - pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); - pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); - } - return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); - } - - private static boolean verify(final PrintStream resultStream, - final String topic, - final Map> inputData, - final Map>>> events, - final Function keyToExpectation, - final boolean printResults) { - final Map>> observedInputEvents = events.get("data"); - final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); - if (outputEvents.isEmpty()) { - resultStream.println(topic + " is empty"); - return false; - } else { - resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); - - if (outputEvents.size() != inputData.size()) { - resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", - outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); - return false; - } - for (final Map.Entry>> entry : outputEvents.entrySet()) { - final String key = entry.getKey(); - final Number expected = keyToExpectation.apply(key); - final Number actual = entry.getValue().getLast().value(); - if (!expected.equals(actual)) { - resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); - - if (printResults) { - resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); - - if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) - resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); - } - - return false; - } - } - return true; - } - } - - - private static boolean verifySuppressed(final PrintStream resultStream, - @SuppressWarnings("SameParameterValue") final String topic, - final Map>>> events, - final boolean printResults) { - resultStream.println("verifying suppressed " + topic); - final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); - for (final Map.Entry>> entry : topicEvents.entrySet()) { - if (entry.getValue().size() != 1) { - final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); - final String key = entry.getKey(); - final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); - resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", - key, - indent("\t\t", entry.getValue())); - - if (printResults) - resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); - - return false; - } - } - return true; - } - - private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, - final Iterable> list) { - final StringBuilder stringBuilder = new StringBuilder(); - for (final ConsumerRecord record : list) { - stringBuilder.append(prefix).append(record).append('\n'); - } - return stringBuilder.toString(); - } - - private static Long getSum(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) * (max - min + 1L) / 2L; - } - - private static Double getAvg(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) / 2.0; - } - - - private static boolean verifyTAgg(final PrintStream resultStream, - final Map> allData, - final Map>> taggEvents, - final boolean printResults) { - if (taggEvents == null) { - resultStream.println("tagg is missing"); - return false; - } else if (taggEvents.isEmpty()) { - resultStream.println("tagg is empty"); - return false; - } else { - resultStream.println("verifying tagg"); - - // generate expected answer - final Map expected = new HashMap<>(); - for (final String key : allData.keySet()) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - final String cnt = Long.toString(max - min + 1L); - - expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); - } - - // check the result - for (final Map.Entry>> entry : taggEvents.entrySet()) { - final String key = entry.getKey(); - Long expectedCount = expected.remove(key); - if (expectedCount == null) { - expectedCount = 0L; - } - - if (entry.getValue().getLast().value().longValue() != expectedCount) { - resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); - - if (printResults) - resultStream.println("\t taggEvents: " + entry.getValue()); - return false; - } - } - - } - return true; - } - - private static Number getMin(final String key) { - return Integer.parseInt(key.split("-")[0]); - } - - private static Number getMax(final String key) { - return Integer.parseInt(key.split("-")[1]); - } - - private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { - final List partitions = new ArrayList<>(); - - for (final String topic : topics) { - for (final PartitionInfo info : consumer.partitionsFor(topic)) { - partitions.add(new TopicPartition(info.topic(), info.partition())); - } - } - return partitions; - } - -} diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java deleted file mode 100644 index e8ec04c23ca52..0000000000000 --- a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -import java.time.Instant; - -public class SmokeTestUtil { - - final static int END = Integer.MAX_VALUE; - - static ProcessorSupplier printProcessorSupplier(final String topic) { - return printProcessorSupplier(topic, ""); - } - - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return new ProcessorSupplier() { - @Override - public Processor get() { - return new AbstractProcessor() { - private int numRecordsProcessed = 0; - private long smallestOffset = Long.MAX_VALUE; - private long largestOffset = Long.MIN_VALUE; - - @Override - public void init(final ProcessorContext context) { - super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); - System.out.flush(); - numRecordsProcessed = 0; - smallestOffset = Long.MAX_VALUE; - largestOffset = Long.MIN_VALUE; - } - - @Override - public void process(final Object key, final Object value) { - numRecordsProcessed++; - if (numRecordsProcessed % 100 == 0) { - System.out.printf("%s: %s%n", name, Instant.now()); - System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); - } - - if (smallestOffset > context().offset()) { - smallestOffset = context().offset(); - } - if (largestOffset < context().offset()) { - largestOffset = context().offset(); - } - } - - @Override - public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); - System.out.println("processed " + numRecordsProcessed + " records"); - final long processed; - if (largestOffset >= smallestOffset) { - processed = 1L + largestOffset - smallestOffset; - } else { - processed = 0L; - } - System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); - System.out.flush(); - } - }; - } - }; - } - - public static final class Unwindow implements KeyValueMapper, V, K> { - @Override - public K apply(final Windowed winKey, final V value) { - return winKey.key(); - } - } - - public static class Agg { - - KeyValueMapper> selector() { - return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); - } - - public Initializer init() { - return () -> 0L; - } - - Aggregator adder() { - return (aggKey, value, aggregate) -> aggregate + value; - } - - Aggregator remover() { - return (aggKey, value, aggregate) -> aggregate - value; - } - } - - public static Serde stringSerde = Serdes.String(); - - public static Serde intSerde = Serdes.Integer(); - - static Serde longSerde = Serdes.Long(); - - static Serde doubleSerde = Serdes.Double(); - - public static void sleep(final long duration) { - try { - Thread.sleep(duration); - } catch (final Exception ignore) { } - } - -} diff --git a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java deleted file mode 100644 index 07c7d5d82ebff..0000000000000 --- a/streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.StreamsConfig; - -import java.io.IOException; -import java.time.Duration; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; - -import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; -import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; - -public class StreamsSmokeTest { - - /** - * args ::= kafka propFileName command disableAutoTerminate - * command := "run" | "process" - * - * @param args - */ - public static void main(final String[] args) throws IOException { - if (args.length < 2) { - System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); - System.exit(1); - } - - final String propFileName = args[0]; - final String command = args[1]; - final boolean disableAutoTerminate = args.length > 2; - - final Properties streamsProperties = Utils.loadProps(propFileName); - final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); - - if (kafka == null) { - System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - System.exit(1); - } - - if ("process".equals(command)) { - if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { - - System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + - StreamsConfig.EXACTLY_ONCE); - - System.exit(1); - } - } - - System.out.println("StreamsTest instance started (StreamsSmokeTest)"); - System.out.println("command=" + command); - System.out.println("props=" + streamsProperties); - System.out.println("disableAutoTerminate=" + disableAutoTerminate); - - switch (command) { - case "run": - // this starts the driver (data generation and result verification) - final int numKeys = 10; - final int maxRecordsPerKey = 500; - if (disableAutoTerminate) { - generatePerpetually(kafka, numKeys, maxRecordsPerKey); - } else { - // slow down data production to span 30 seconds so that system tests have time to - // do their bounces, etc. - final Map> allData = - generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); - SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); - } - break; - case "process": - // this starts the stream processing app - new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); - break; - default: - System.out.println("unknown command: " + command); - } - } - -} diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java deleted file mode 100644 index ced1369af3855..0000000000000 --- a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ /dev/null @@ -1,298 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.Topology; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Grouped; -import org.apache.kafka.streams.kstream.KGroupedStream; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; -import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.streams.state.WindowStore; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; - -public class SmokeTestClient extends SmokeTestUtil { - - private final String name; - - private KafkaStreams streams; - private boolean uncaughtException = false; - private boolean started; - private volatile boolean closed; - - private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { - Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); - } else { - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } - } - - private static File tempDirectory() { - final String prefix = "kafka-"; - final File file; - try { - file = Files.createTempDirectory(prefix).toFile(); - } catch (final IOException ex) { - throw new RuntimeException("Failed to create a temp dir", ex); - } - file.deleteOnExit(); - - addShutdownHook("delete-temp-file-shutdown-hook", () -> { - try { - Utils.delete(file); - } catch (final IOException e) { - System.out.println("Error deleting " + file.getAbsolutePath()); - e.printStackTrace(System.out); - } - }); - - return file; - } - - public SmokeTestClient(final String name) { - this.name = name; - } - - public boolean started() { - return started; - } - - public boolean closed() { - return closed; - } - - public void start(final Properties streamsProperties) { - final Topology build = getTopology(); - streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); - - final CountDownLatch countDownLatch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; - countDownLatch.countDown(); - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - - streams.setUncaughtExceptionHandler((t, e) -> { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - e.printStackTrace(System.out); - uncaughtException = true; - streams.close(Duration.ofSeconds(30)); - }); - - addShutdownHook("streams-shutdown-hook", this::close); - - streams.start(); - try { - if (!countDownLatch.await(1, TimeUnit.MINUTES)) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); - } - } catch (final InterruptedException e) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); - e.printStackTrace(System.out); - } - System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); - System.out.println(name + " started at " + Instant.now()); - } - - public void closeAsync() { - streams.close(Duration.ZERO); - } - - public void close() { - final boolean closed = streams.close(Duration.ofMinutes(1)); - - if (closed && !uncaughtException) { - System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } else if (closed) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - } else { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); - } - } - - private Properties getStreamsConfig(final Properties props) { - final Properties fullProps = new Properties(props); - fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); - fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); - fullProps.putAll(props); - return fullProps; - } - - public Topology getTopology() { - final StreamsBuilder builder = new StreamsBuilder(); - final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); - final KStream source = builder.stream("data", stringIntConsumed); - source.filterNot((k, v) -> k.equals("flush")) - .to("echo", Produced.with(stringSerde, intSerde)); - final KStream data = source.filter((key, value) -> value == null || value != END); - data.process(SmokeTestUtil.printProcessorSupplier("data", name)); - - // min - final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); - - final KTable, Integer> minAggregation = groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) - .aggregate( - () -> Integer.MAX_VALUE, - (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, - Materialized - .>as("uwin-min") - .withValueSerde(intSerde) - .withRetention(Duration.ofHours(25)) - ); - - streamify(minAggregation, "min-raw"); - - streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); - - minAggregation - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("min", Produced.with(stringSerde, intSerde)); - - final KTable, Integer> smallWindowSum = groupedData - .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) - .reduce((l, r) -> l + r); - - streamify(smallWindowSum, "sws-raw"); - streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); - - final KTable minTable = builder.table( - "min", - Consumed.with(stringSerde, intSerde), - Materialized.as("minStoreName")); - - minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); - - // max - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> Integer.MIN_VALUE, - (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, - Materialized.>as("uwin-max").withValueSerde(intSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("max", Produced.with(stringSerde, intSerde)); - - final KTable maxTable = builder.table( - "max", - Consumed.with(stringSerde, intSerde), - Materialized.as("maxStoreName")); - maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); - - // sum - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> 0L, - (aggKey, value, aggregate) -> (long) value + aggregate, - Materialized.>as("win-sum").withValueSerde(longSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("sum", Produced.with(stringSerde, longSerde)); - - final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); - final KTable sumTable = builder.table("sum", stringLongConsumed); - sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); - - // cnt - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .count(Materialized.as("uwin-cnt")) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("cnt", Produced.with(stringSerde, longSerde)); - - final KTable cntTable = builder.table( - "cnt", - Consumed.with(stringSerde, longSerde), - Materialized.as("cntStoreName")); - cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); - - // dif - maxTable - .join( - minTable, - (value1, value2) -> value1 - value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("dif", Produced.with(stringSerde, intSerde)); - - // avg - sumTable - .join( - cntTable, - (value1, value2) -> (double) value1 / (double) value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("avg", Produced.with(stringSerde, doubleSerde)); - - // test repartition - final Agg agg = new Agg(); - cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) - .aggregate(agg.init(), agg.adder(), agg.remover(), - Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) - .withKeySerde(Serdes.String()) - .withValueSerde(Serdes.Long())) - .toStream() - .to("tagg", Produced.with(stringSerde, longSerde)); - - return builder.build(); - } - - private static void streamify(final KTable, Integer> windowedTable, final String topic) { - windowedTable - .toStream() - .filterNot((k, v) -> k.key().equals("flush")) - .map((key, value) -> new KeyValue<>(key.toString(), value)) - .to(topic, Produced.with(stringSerde, intSerde)); - } -} diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java deleted file mode 100644 index ac83cd95ebaaf..0000000000000 --- a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ /dev/null @@ -1,622 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.utils.Utils; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; - -public class SmokeTestDriver extends SmokeTestUtil { - private static final String[] TOPICS = { - "data", - "echo", - "max", - "min", "min-suppressed", "min-raw", - "dif", - "sum", - "sws-raw", "sws-suppressed", - "cnt", - "avg", - "tagg" - }; - - private static final int MAX_RECORD_EMPTY_RETRIES = 30; - - private static class ValueList { - public final String key; - private final int[] values; - private int index; - - ValueList(final int min, final int max) { - key = min + "-" + max; - - values = new int[max - min + 1]; - for (int i = 0; i < values.length; i++) { - values[i] = min + i; - } - // We want to randomize the order of data to test not completely predictable processing order - // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) - // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window - shuffle(values, 10); - - index = 0; - } - - int next() { - return (index < values.length) ? values[index++] : -1; - } - } - - public static String[] topics() { - return Arrays.copyOf(TOPICS, TOPICS.length); - } - - static void generatePerpetually(final String kafka, - final int numKeys, - final int maxRecordsPerKey) { - final Properties producerProps = generatorProperties(kafka); - - int numRecordsProduced = 0; - - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - } - - final Random rand = new Random(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (true) { - final int index = rand.nextInt(numKeys); - final String key = data[index].key; - final int value = data[index].next(); - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record); - - numRecordsProduced++; - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(2); - } - } - } - - public static Map> generate(final String kafka, - final int numKeys, - final int maxRecordsPerKey, - final Duration timeToSpend) { - final Properties producerProps = generatorProperties(kafka); - - - int numRecordsProduced = 0; - - final Map> allData = new HashMap<>(); - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - allData.put(data[i].key, new HashSet<>()); - } - final Random rand = new Random(); - - int remaining = data.length; - - final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; - - List> needRetry = new ArrayList<>(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (remaining > 0) { - final int index = rand.nextInt(remaining); - final String key = data[index].key; - final int value = data[index].next(); - - if (value < 0) { - remaining--; - data[index] = data[remaining]; - } else { - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record, new TestCallback(record, needRetry)); - - numRecordsProduced++; - allData.get(key).add(value); - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(Math.max(recordPauseTime, 2)); - } - } - producer.flush(); - - int remainingRetries = 5; - while (!needRetry.isEmpty()) { - final List> needRetry2 = new ArrayList<>(); - for (final ProducerRecord record : needRetry) { - System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); - producer.send(record, new TestCallback(record, needRetry2)); - } - producer.flush(); - needRetry = needRetry2; - - if (--remainingRetries == 0 && !needRetry.isEmpty()) { - System.err.println("Failed to produce all records after multiple retries"); - Exit.exit(1); - } - } - - // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out - // all suppressed records. - final List partitions = producer.partitionsFor("data"); - for (final PartitionInfo partition : partitions) { - producer.send(new ProducerRecord<>( - partition.topic(), - partition.partition(), - System.currentTimeMillis() + Duration.ofDays(2).toMillis(), - stringSerde.serializer().serialize("", "flush"), - intSerde.serializer().serialize("", 0) - )); - } - } - return Collections.unmodifiableMap(allData); - } - - private static Properties generatorProperties(final String kafka) { - final Properties producerProps = new Properties(); - producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); - return producerProps; - } - - private static class TestCallback implements Callback { - private final ProducerRecord originalRecord; - private final List> needRetry; - - TestCallback(final ProducerRecord originalRecord, - final List> needRetry) { - this.originalRecord = originalRecord; - this.needRetry = needRetry; - } - - @Override - public void onCompletion(final RecordMetadata metadata, final Exception exception) { - if (exception != null) { - if (exception instanceof TimeoutException) { - needRetry.add(originalRecord); - } else { - exception.printStackTrace(); - Exit.exit(1); - } - } - } - } - - private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { - final Random rand = new Random(); - for (int i = 0; i < data.length; i++) { - // we shuffle data within windowSize - final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; - - // swap - final int tmp = data[i]; - data[i] = data[j]; - data[j] = tmp; - } - } - - public static class NumberDeserializer implements Deserializer { - @Override - public Number deserialize(final String topic, final byte[] data) { - final Number value; - switch (topic) { - case "data": - case "echo": - case "min": - case "min-raw": - case "min-suppressed": - case "sws-raw": - case "sws-suppressed": - case "max": - case "dif": - value = intSerde.deserializer().deserialize(topic, data); - break; - case "sum": - case "cnt": - case "tagg": - value = longSerde.deserializer().deserialize(topic, data); - break; - case "avg": - value = doubleSerde.deserializer().deserialize(topic, data); - break; - default: - throw new RuntimeException("unknown topic: " + topic); - } - return value; - } - } - - public static VerificationResult verify(final String kafka, - final Map> inputs, - final int maxRecordsPerKey) { - final Properties props = new Properties(); - props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); - props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); - - final KafkaConsumer consumer = new KafkaConsumer<>(props); - final List partitions = getAllPartitions(consumer, TOPICS); - consumer.assign(partitions); - consumer.seekToBeginning(partitions); - - final int recordsGenerated = inputs.size() * maxRecordsPerKey; - int recordsProcessed = 0; - final Map processed = - Stream.of(TOPICS) - .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); - - final Map>>> events = new HashMap<>(); - - VerificationResult verificationResult = new VerificationResult(false, "no results yet"); - int retry = 0; - final long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { - final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (records.isEmpty() && recordsProcessed >= recordsGenerated) { - verificationResult = verifyAll(inputs, events, false); - if (verificationResult.passed()) { - break; - } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { - System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); - break; - } else { - System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); - } - } else { - System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); - - retry = 0; - for (final ConsumerRecord record : records) { - final String key = record.key(); - - final String topic = record.topic(); - processed.get(topic).incrementAndGet(); - - if (topic.equals("echo")) { - recordsProcessed++; - if (recordsProcessed % 100 == 0) { - System.out.println("Echo records processed = " + recordsProcessed); - } - } - - events.computeIfAbsent(topic, t -> new HashMap<>()) - .computeIfAbsent(key, k -> new LinkedList<>()) - .add(record); - } - - System.out.println(processed); - } - } - consumer.close(); - final long finished = System.currentTimeMillis() - start; - System.out.println("Verification time=" + finished); - System.out.println("-------------------"); - System.out.println("Result Verification"); - System.out.println("-------------------"); - System.out.println("recordGenerated=" + recordsGenerated); - System.out.println("recordProcessed=" + recordsProcessed); - - if (recordsProcessed > recordsGenerated) { - System.out.println("PROCESSED-MORE-THAN-GENERATED"); - } else if (recordsProcessed < recordsGenerated) { - System.out.println("PROCESSED-LESS-THAN-GENERATED"); - } - - boolean success; - - final Map> received = - events.get("echo") - .entrySet() - .stream() - .map(entry -> mkEntry( - entry.getKey(), - entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) - ) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - success = inputs.equals(received); - - if (success) { - System.out.println("ALL-RECORDS-DELIVERED"); - } else { - int missedCount = 0; - for (final Map.Entry> entry : inputs.entrySet()) { - missedCount += received.get(entry.getKey()).size(); - } - System.out.println("missedRecords=" + missedCount); - } - - // give it one more try if it's not already passing. - if (!verificationResult.passed()) { - verificationResult = verifyAll(inputs, events, true); - } - success &= verificationResult.passed(); - - System.out.println(verificationResult.result()); - - System.out.println(success ? "SUCCESS" : "FAILURE"); - return verificationResult; - } - - public static class VerificationResult { - private final boolean passed; - private final String result; - - VerificationResult(final boolean passed, final String result) { - this.passed = passed; - this.result = result; - } - - public boolean passed() { - return passed; - } - - public String result() { - return result; - } - } - - private static VerificationResult verifyAll(final Map> inputs, - final Map>>> events, - final boolean printResults) { - final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - boolean pass; - try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { - pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); - pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); - pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { - final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); - return getMin(unwindowedKey); - }, printResults); - pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); - pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); - pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); - pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); - pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); - pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); - pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); - } - return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); - } - - private static boolean verify(final PrintStream resultStream, - final String topic, - final Map> inputData, - final Map>>> events, - final Function keyToExpectation, - final boolean printResults) { - final Map>> observedInputEvents = events.get("data"); - final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); - if (outputEvents.isEmpty()) { - resultStream.println(topic + " is empty"); - return false; - } else { - resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); - - if (outputEvents.size() != inputData.size()) { - resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", - outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); - return false; - } - for (final Map.Entry>> entry : outputEvents.entrySet()) { - final String key = entry.getKey(); - final Number expected = keyToExpectation.apply(key); - final Number actual = entry.getValue().getLast().value(); - if (!expected.equals(actual)) { - resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); - - if (printResults) { - resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); - - if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) - resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); - } - - return false; - } - } - return true; - } - } - - - private static boolean verifySuppressed(final PrintStream resultStream, - @SuppressWarnings("SameParameterValue") final String topic, - final Map>>> events, - final boolean printResults) { - resultStream.println("verifying suppressed " + topic); - final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); - for (final Map.Entry>> entry : topicEvents.entrySet()) { - if (entry.getValue().size() != 1) { - final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); - final String key = entry.getKey(); - final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); - resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", - key, - indent("\t\t", entry.getValue())); - - if (printResults) - resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); - - return false; - } - } - return true; - } - - private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, - final Iterable> list) { - final StringBuilder stringBuilder = new StringBuilder(); - for (final ConsumerRecord record : list) { - stringBuilder.append(prefix).append(record).append('\n'); - } - return stringBuilder.toString(); - } - - private static Long getSum(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) * (max - min + 1L) / 2L; - } - - private static Double getAvg(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) / 2.0; - } - - - private static boolean verifyTAgg(final PrintStream resultStream, - final Map> allData, - final Map>> taggEvents, - final boolean printResults) { - if (taggEvents == null) { - resultStream.println("tagg is missing"); - return false; - } else if (taggEvents.isEmpty()) { - resultStream.println("tagg is empty"); - return false; - } else { - resultStream.println("verifying tagg"); - - // generate expected answer - final Map expected = new HashMap<>(); - for (final String key : allData.keySet()) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - final String cnt = Long.toString(max - min + 1L); - - expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); - } - - // check the result - for (final Map.Entry>> entry : taggEvents.entrySet()) { - final String key = entry.getKey(); - Long expectedCount = expected.remove(key); - if (expectedCount == null) { - expectedCount = 0L; - } - - if (entry.getValue().getLast().value().longValue() != expectedCount) { - resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); - - if (printResults) - resultStream.println("\t taggEvents: " + entry.getValue()); - return false; - } - } - - } - return true; - } - - private static Number getMin(final String key) { - return Integer.parseInt(key.split("-")[0]); - } - - private static Number getMax(final String key) { - return Integer.parseInt(key.split("-")[1]); - } - - private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { - final List partitions = new ArrayList<>(); - - for (final String topic : topics) { - for (final PartitionInfo info : consumer.partitionsFor(topic)) { - partitions.add(new TopicPartition(info.topic(), info.partition())); - } - } - return partitions; - } - -} diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java deleted file mode 100644 index e8ec04c23ca52..0000000000000 --- a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -import java.time.Instant; - -public class SmokeTestUtil { - - final static int END = Integer.MAX_VALUE; - - static ProcessorSupplier printProcessorSupplier(final String topic) { - return printProcessorSupplier(topic, ""); - } - - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return new ProcessorSupplier() { - @Override - public Processor get() { - return new AbstractProcessor() { - private int numRecordsProcessed = 0; - private long smallestOffset = Long.MAX_VALUE; - private long largestOffset = Long.MIN_VALUE; - - @Override - public void init(final ProcessorContext context) { - super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); - System.out.flush(); - numRecordsProcessed = 0; - smallestOffset = Long.MAX_VALUE; - largestOffset = Long.MIN_VALUE; - } - - @Override - public void process(final Object key, final Object value) { - numRecordsProcessed++; - if (numRecordsProcessed % 100 == 0) { - System.out.printf("%s: %s%n", name, Instant.now()); - System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); - } - - if (smallestOffset > context().offset()) { - smallestOffset = context().offset(); - } - if (largestOffset < context().offset()) { - largestOffset = context().offset(); - } - } - - @Override - public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); - System.out.println("processed " + numRecordsProcessed + " records"); - final long processed; - if (largestOffset >= smallestOffset) { - processed = 1L + largestOffset - smallestOffset; - } else { - processed = 0L; - } - System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); - System.out.flush(); - } - }; - } - }; - } - - public static final class Unwindow implements KeyValueMapper, V, K> { - @Override - public K apply(final Windowed winKey, final V value) { - return winKey.key(); - } - } - - public static class Agg { - - KeyValueMapper> selector() { - return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); - } - - public Initializer init() { - return () -> 0L; - } - - Aggregator adder() { - return (aggKey, value, aggregate) -> aggregate + value; - } - - Aggregator remover() { - return (aggKey, value, aggregate) -> aggregate - value; - } - } - - public static Serde stringSerde = Serdes.String(); - - public static Serde intSerde = Serdes.Integer(); - - static Serde longSerde = Serdes.Long(); - - static Serde doubleSerde = Serdes.Double(); - - public static void sleep(final long duration) { - try { - Thread.sleep(duration); - } catch (final Exception ignore) { } - } - -} diff --git a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java deleted file mode 100644 index 07c7d5d82ebff..0000000000000 --- a/streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.StreamsConfig; - -import java.io.IOException; -import java.time.Duration; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; - -import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; -import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; - -public class StreamsSmokeTest { - - /** - * args ::= kafka propFileName command disableAutoTerminate - * command := "run" | "process" - * - * @param args - */ - public static void main(final String[] args) throws IOException { - if (args.length < 2) { - System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); - System.exit(1); - } - - final String propFileName = args[0]; - final String command = args[1]; - final boolean disableAutoTerminate = args.length > 2; - - final Properties streamsProperties = Utils.loadProps(propFileName); - final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); - - if (kafka == null) { - System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - System.exit(1); - } - - if ("process".equals(command)) { - if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { - - System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + - StreamsConfig.EXACTLY_ONCE); - - System.exit(1); - } - } - - System.out.println("StreamsTest instance started (StreamsSmokeTest)"); - System.out.println("command=" + command); - System.out.println("props=" + streamsProperties); - System.out.println("disableAutoTerminate=" + disableAutoTerminate); - - switch (command) { - case "run": - // this starts the driver (data generation and result verification) - final int numKeys = 10; - final int maxRecordsPerKey = 500; - if (disableAutoTerminate) { - generatePerpetually(kafka, numKeys, maxRecordsPerKey); - } else { - // slow down data production to span 30 seconds so that system tests have time to - // do their bounces, etc. - final Map> allData = - generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); - SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); - } - break; - case "process": - // this starts the stream processing app - new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); - break; - default: - System.out.println("unknown command: " + command); - } - } - -} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java deleted file mode 100644 index ced1369af3855..0000000000000 --- a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ /dev/null @@ -1,298 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.Topology; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Grouped; -import org.apache.kafka.streams.kstream.KGroupedStream; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; -import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.streams.state.WindowStore; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; - -public class SmokeTestClient extends SmokeTestUtil { - - private final String name; - - private KafkaStreams streams; - private boolean uncaughtException = false; - private boolean started; - private volatile boolean closed; - - private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { - Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); - } else { - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } - } - - private static File tempDirectory() { - final String prefix = "kafka-"; - final File file; - try { - file = Files.createTempDirectory(prefix).toFile(); - } catch (final IOException ex) { - throw new RuntimeException("Failed to create a temp dir", ex); - } - file.deleteOnExit(); - - addShutdownHook("delete-temp-file-shutdown-hook", () -> { - try { - Utils.delete(file); - } catch (final IOException e) { - System.out.println("Error deleting " + file.getAbsolutePath()); - e.printStackTrace(System.out); - } - }); - - return file; - } - - public SmokeTestClient(final String name) { - this.name = name; - } - - public boolean started() { - return started; - } - - public boolean closed() { - return closed; - } - - public void start(final Properties streamsProperties) { - final Topology build = getTopology(); - streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); - - final CountDownLatch countDownLatch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; - countDownLatch.countDown(); - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - - streams.setUncaughtExceptionHandler((t, e) -> { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - e.printStackTrace(System.out); - uncaughtException = true; - streams.close(Duration.ofSeconds(30)); - }); - - addShutdownHook("streams-shutdown-hook", this::close); - - streams.start(); - try { - if (!countDownLatch.await(1, TimeUnit.MINUTES)) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); - } - } catch (final InterruptedException e) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); - e.printStackTrace(System.out); - } - System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); - System.out.println(name + " started at " + Instant.now()); - } - - public void closeAsync() { - streams.close(Duration.ZERO); - } - - public void close() { - final boolean closed = streams.close(Duration.ofMinutes(1)); - - if (closed && !uncaughtException) { - System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } else if (closed) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - } else { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); - } - } - - private Properties getStreamsConfig(final Properties props) { - final Properties fullProps = new Properties(props); - fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); - fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); - fullProps.putAll(props); - return fullProps; - } - - public Topology getTopology() { - final StreamsBuilder builder = new StreamsBuilder(); - final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); - final KStream source = builder.stream("data", stringIntConsumed); - source.filterNot((k, v) -> k.equals("flush")) - .to("echo", Produced.with(stringSerde, intSerde)); - final KStream data = source.filter((key, value) -> value == null || value != END); - data.process(SmokeTestUtil.printProcessorSupplier("data", name)); - - // min - final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); - - final KTable, Integer> minAggregation = groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) - .aggregate( - () -> Integer.MAX_VALUE, - (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, - Materialized - .>as("uwin-min") - .withValueSerde(intSerde) - .withRetention(Duration.ofHours(25)) - ); - - streamify(minAggregation, "min-raw"); - - streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); - - minAggregation - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("min", Produced.with(stringSerde, intSerde)); - - final KTable, Integer> smallWindowSum = groupedData - .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) - .reduce((l, r) -> l + r); - - streamify(smallWindowSum, "sws-raw"); - streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); - - final KTable minTable = builder.table( - "min", - Consumed.with(stringSerde, intSerde), - Materialized.as("minStoreName")); - - minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); - - // max - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> Integer.MIN_VALUE, - (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, - Materialized.>as("uwin-max").withValueSerde(intSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("max", Produced.with(stringSerde, intSerde)); - - final KTable maxTable = builder.table( - "max", - Consumed.with(stringSerde, intSerde), - Materialized.as("maxStoreName")); - maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); - - // sum - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> 0L, - (aggKey, value, aggregate) -> (long) value + aggregate, - Materialized.>as("win-sum").withValueSerde(longSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("sum", Produced.with(stringSerde, longSerde)); - - final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); - final KTable sumTable = builder.table("sum", stringLongConsumed); - sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); - - // cnt - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .count(Materialized.as("uwin-cnt")) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("cnt", Produced.with(stringSerde, longSerde)); - - final KTable cntTable = builder.table( - "cnt", - Consumed.with(stringSerde, longSerde), - Materialized.as("cntStoreName")); - cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); - - // dif - maxTable - .join( - minTable, - (value1, value2) -> value1 - value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("dif", Produced.with(stringSerde, intSerde)); - - // avg - sumTable - .join( - cntTable, - (value1, value2) -> (double) value1 / (double) value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("avg", Produced.with(stringSerde, doubleSerde)); - - // test repartition - final Agg agg = new Agg(); - cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) - .aggregate(agg.init(), agg.adder(), agg.remover(), - Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) - .withKeySerde(Serdes.String()) - .withValueSerde(Serdes.Long())) - .toStream() - .to("tagg", Produced.with(stringSerde, longSerde)); - - return builder.build(); - } - - private static void streamify(final KTable, Integer> windowedTable, final String topic) { - windowedTable - .toStream() - .filterNot((k, v) -> k.key().equals("flush")) - .map((key, value) -> new KeyValue<>(key.toString(), value)) - .to(topic, Produced.with(stringSerde, intSerde)); - } -} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java deleted file mode 100644 index ac83cd95ebaaf..0000000000000 --- a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ /dev/null @@ -1,622 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.utils.Utils; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; - -public class SmokeTestDriver extends SmokeTestUtil { - private static final String[] TOPICS = { - "data", - "echo", - "max", - "min", "min-suppressed", "min-raw", - "dif", - "sum", - "sws-raw", "sws-suppressed", - "cnt", - "avg", - "tagg" - }; - - private static final int MAX_RECORD_EMPTY_RETRIES = 30; - - private static class ValueList { - public final String key; - private final int[] values; - private int index; - - ValueList(final int min, final int max) { - key = min + "-" + max; - - values = new int[max - min + 1]; - for (int i = 0; i < values.length; i++) { - values[i] = min + i; - } - // We want to randomize the order of data to test not completely predictable processing order - // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) - // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window - shuffle(values, 10); - - index = 0; - } - - int next() { - return (index < values.length) ? values[index++] : -1; - } - } - - public static String[] topics() { - return Arrays.copyOf(TOPICS, TOPICS.length); - } - - static void generatePerpetually(final String kafka, - final int numKeys, - final int maxRecordsPerKey) { - final Properties producerProps = generatorProperties(kafka); - - int numRecordsProduced = 0; - - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - } - - final Random rand = new Random(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (true) { - final int index = rand.nextInt(numKeys); - final String key = data[index].key; - final int value = data[index].next(); - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record); - - numRecordsProduced++; - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(2); - } - } - } - - public static Map> generate(final String kafka, - final int numKeys, - final int maxRecordsPerKey, - final Duration timeToSpend) { - final Properties producerProps = generatorProperties(kafka); - - - int numRecordsProduced = 0; - - final Map> allData = new HashMap<>(); - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - allData.put(data[i].key, new HashSet<>()); - } - final Random rand = new Random(); - - int remaining = data.length; - - final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; - - List> needRetry = new ArrayList<>(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (remaining > 0) { - final int index = rand.nextInt(remaining); - final String key = data[index].key; - final int value = data[index].next(); - - if (value < 0) { - remaining--; - data[index] = data[remaining]; - } else { - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record, new TestCallback(record, needRetry)); - - numRecordsProduced++; - allData.get(key).add(value); - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(Math.max(recordPauseTime, 2)); - } - } - producer.flush(); - - int remainingRetries = 5; - while (!needRetry.isEmpty()) { - final List> needRetry2 = new ArrayList<>(); - for (final ProducerRecord record : needRetry) { - System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); - producer.send(record, new TestCallback(record, needRetry2)); - } - producer.flush(); - needRetry = needRetry2; - - if (--remainingRetries == 0 && !needRetry.isEmpty()) { - System.err.println("Failed to produce all records after multiple retries"); - Exit.exit(1); - } - } - - // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out - // all suppressed records. - final List partitions = producer.partitionsFor("data"); - for (final PartitionInfo partition : partitions) { - producer.send(new ProducerRecord<>( - partition.topic(), - partition.partition(), - System.currentTimeMillis() + Duration.ofDays(2).toMillis(), - stringSerde.serializer().serialize("", "flush"), - intSerde.serializer().serialize("", 0) - )); - } - } - return Collections.unmodifiableMap(allData); - } - - private static Properties generatorProperties(final String kafka) { - final Properties producerProps = new Properties(); - producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); - return producerProps; - } - - private static class TestCallback implements Callback { - private final ProducerRecord originalRecord; - private final List> needRetry; - - TestCallback(final ProducerRecord originalRecord, - final List> needRetry) { - this.originalRecord = originalRecord; - this.needRetry = needRetry; - } - - @Override - public void onCompletion(final RecordMetadata metadata, final Exception exception) { - if (exception != null) { - if (exception instanceof TimeoutException) { - needRetry.add(originalRecord); - } else { - exception.printStackTrace(); - Exit.exit(1); - } - } - } - } - - private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { - final Random rand = new Random(); - for (int i = 0; i < data.length; i++) { - // we shuffle data within windowSize - final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; - - // swap - final int tmp = data[i]; - data[i] = data[j]; - data[j] = tmp; - } - } - - public static class NumberDeserializer implements Deserializer { - @Override - public Number deserialize(final String topic, final byte[] data) { - final Number value; - switch (topic) { - case "data": - case "echo": - case "min": - case "min-raw": - case "min-suppressed": - case "sws-raw": - case "sws-suppressed": - case "max": - case "dif": - value = intSerde.deserializer().deserialize(topic, data); - break; - case "sum": - case "cnt": - case "tagg": - value = longSerde.deserializer().deserialize(topic, data); - break; - case "avg": - value = doubleSerde.deserializer().deserialize(topic, data); - break; - default: - throw new RuntimeException("unknown topic: " + topic); - } - return value; - } - } - - public static VerificationResult verify(final String kafka, - final Map> inputs, - final int maxRecordsPerKey) { - final Properties props = new Properties(); - props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); - props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); - - final KafkaConsumer consumer = new KafkaConsumer<>(props); - final List partitions = getAllPartitions(consumer, TOPICS); - consumer.assign(partitions); - consumer.seekToBeginning(partitions); - - final int recordsGenerated = inputs.size() * maxRecordsPerKey; - int recordsProcessed = 0; - final Map processed = - Stream.of(TOPICS) - .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); - - final Map>>> events = new HashMap<>(); - - VerificationResult verificationResult = new VerificationResult(false, "no results yet"); - int retry = 0; - final long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { - final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (records.isEmpty() && recordsProcessed >= recordsGenerated) { - verificationResult = verifyAll(inputs, events, false); - if (verificationResult.passed()) { - break; - } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { - System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); - break; - } else { - System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); - } - } else { - System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); - - retry = 0; - for (final ConsumerRecord record : records) { - final String key = record.key(); - - final String topic = record.topic(); - processed.get(topic).incrementAndGet(); - - if (topic.equals("echo")) { - recordsProcessed++; - if (recordsProcessed % 100 == 0) { - System.out.println("Echo records processed = " + recordsProcessed); - } - } - - events.computeIfAbsent(topic, t -> new HashMap<>()) - .computeIfAbsent(key, k -> new LinkedList<>()) - .add(record); - } - - System.out.println(processed); - } - } - consumer.close(); - final long finished = System.currentTimeMillis() - start; - System.out.println("Verification time=" + finished); - System.out.println("-------------------"); - System.out.println("Result Verification"); - System.out.println("-------------------"); - System.out.println("recordGenerated=" + recordsGenerated); - System.out.println("recordProcessed=" + recordsProcessed); - - if (recordsProcessed > recordsGenerated) { - System.out.println("PROCESSED-MORE-THAN-GENERATED"); - } else if (recordsProcessed < recordsGenerated) { - System.out.println("PROCESSED-LESS-THAN-GENERATED"); - } - - boolean success; - - final Map> received = - events.get("echo") - .entrySet() - .stream() - .map(entry -> mkEntry( - entry.getKey(), - entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) - ) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - success = inputs.equals(received); - - if (success) { - System.out.println("ALL-RECORDS-DELIVERED"); - } else { - int missedCount = 0; - for (final Map.Entry> entry : inputs.entrySet()) { - missedCount += received.get(entry.getKey()).size(); - } - System.out.println("missedRecords=" + missedCount); - } - - // give it one more try if it's not already passing. - if (!verificationResult.passed()) { - verificationResult = verifyAll(inputs, events, true); - } - success &= verificationResult.passed(); - - System.out.println(verificationResult.result()); - - System.out.println(success ? "SUCCESS" : "FAILURE"); - return verificationResult; - } - - public static class VerificationResult { - private final boolean passed; - private final String result; - - VerificationResult(final boolean passed, final String result) { - this.passed = passed; - this.result = result; - } - - public boolean passed() { - return passed; - } - - public String result() { - return result; - } - } - - private static VerificationResult verifyAll(final Map> inputs, - final Map>>> events, - final boolean printResults) { - final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - boolean pass; - try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { - pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); - pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); - pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { - final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); - return getMin(unwindowedKey); - }, printResults); - pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); - pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); - pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); - pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); - pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); - pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); - pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); - } - return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); - } - - private static boolean verify(final PrintStream resultStream, - final String topic, - final Map> inputData, - final Map>>> events, - final Function keyToExpectation, - final boolean printResults) { - final Map>> observedInputEvents = events.get("data"); - final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); - if (outputEvents.isEmpty()) { - resultStream.println(topic + " is empty"); - return false; - } else { - resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); - - if (outputEvents.size() != inputData.size()) { - resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", - outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); - return false; - } - for (final Map.Entry>> entry : outputEvents.entrySet()) { - final String key = entry.getKey(); - final Number expected = keyToExpectation.apply(key); - final Number actual = entry.getValue().getLast().value(); - if (!expected.equals(actual)) { - resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); - - if (printResults) { - resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); - - if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) - resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); - } - - return false; - } - } - return true; - } - } - - - private static boolean verifySuppressed(final PrintStream resultStream, - @SuppressWarnings("SameParameterValue") final String topic, - final Map>>> events, - final boolean printResults) { - resultStream.println("verifying suppressed " + topic); - final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); - for (final Map.Entry>> entry : topicEvents.entrySet()) { - if (entry.getValue().size() != 1) { - final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); - final String key = entry.getKey(); - final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); - resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", - key, - indent("\t\t", entry.getValue())); - - if (printResults) - resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); - - return false; - } - } - return true; - } - - private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, - final Iterable> list) { - final StringBuilder stringBuilder = new StringBuilder(); - for (final ConsumerRecord record : list) { - stringBuilder.append(prefix).append(record).append('\n'); - } - return stringBuilder.toString(); - } - - private static Long getSum(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) * (max - min + 1L) / 2L; - } - - private static Double getAvg(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) / 2.0; - } - - - private static boolean verifyTAgg(final PrintStream resultStream, - final Map> allData, - final Map>> taggEvents, - final boolean printResults) { - if (taggEvents == null) { - resultStream.println("tagg is missing"); - return false; - } else if (taggEvents.isEmpty()) { - resultStream.println("tagg is empty"); - return false; - } else { - resultStream.println("verifying tagg"); - - // generate expected answer - final Map expected = new HashMap<>(); - for (final String key : allData.keySet()) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - final String cnt = Long.toString(max - min + 1L); - - expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); - } - - // check the result - for (final Map.Entry>> entry : taggEvents.entrySet()) { - final String key = entry.getKey(); - Long expectedCount = expected.remove(key); - if (expectedCount == null) { - expectedCount = 0L; - } - - if (entry.getValue().getLast().value().longValue() != expectedCount) { - resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); - - if (printResults) - resultStream.println("\t taggEvents: " + entry.getValue()); - return false; - } - } - - } - return true; - } - - private static Number getMin(final String key) { - return Integer.parseInt(key.split("-")[0]); - } - - private static Number getMax(final String key) { - return Integer.parseInt(key.split("-")[1]); - } - - private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { - final List partitions = new ArrayList<>(); - - for (final String topic : topics) { - for (final PartitionInfo info : consumer.partitionsFor(topic)) { - partitions.add(new TopicPartition(info.topic(), info.partition())); - } - } - return partitions; - } - -} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java deleted file mode 100644 index e8ec04c23ca52..0000000000000 --- a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -import java.time.Instant; - -public class SmokeTestUtil { - - final static int END = Integer.MAX_VALUE; - - static ProcessorSupplier printProcessorSupplier(final String topic) { - return printProcessorSupplier(topic, ""); - } - - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return new ProcessorSupplier() { - @Override - public Processor get() { - return new AbstractProcessor() { - private int numRecordsProcessed = 0; - private long smallestOffset = Long.MAX_VALUE; - private long largestOffset = Long.MIN_VALUE; - - @Override - public void init(final ProcessorContext context) { - super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); - System.out.flush(); - numRecordsProcessed = 0; - smallestOffset = Long.MAX_VALUE; - largestOffset = Long.MIN_VALUE; - } - - @Override - public void process(final Object key, final Object value) { - numRecordsProcessed++; - if (numRecordsProcessed % 100 == 0) { - System.out.printf("%s: %s%n", name, Instant.now()); - System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); - } - - if (smallestOffset > context().offset()) { - smallestOffset = context().offset(); - } - if (largestOffset < context().offset()) { - largestOffset = context().offset(); - } - } - - @Override - public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); - System.out.println("processed " + numRecordsProcessed + " records"); - final long processed; - if (largestOffset >= smallestOffset) { - processed = 1L + largestOffset - smallestOffset; - } else { - processed = 0L; - } - System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); - System.out.flush(); - } - }; - } - }; - } - - public static final class Unwindow implements KeyValueMapper, V, K> { - @Override - public K apply(final Windowed winKey, final V value) { - return winKey.key(); - } - } - - public static class Agg { - - KeyValueMapper> selector() { - return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); - } - - public Initializer init() { - return () -> 0L; - } - - Aggregator adder() { - return (aggKey, value, aggregate) -> aggregate + value; - } - - Aggregator remover() { - return (aggKey, value, aggregate) -> aggregate - value; - } - } - - public static Serde stringSerde = Serdes.String(); - - public static Serde intSerde = Serdes.Integer(); - - static Serde longSerde = Serdes.Long(); - - static Serde doubleSerde = Serdes.Double(); - - public static void sleep(final long duration) { - try { - Thread.sleep(duration); - } catch (final Exception ignore) { } - } - -} diff --git a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java deleted file mode 100644 index 07c7d5d82ebff..0000000000000 --- a/streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.StreamsConfig; - -import java.io.IOException; -import java.time.Duration; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; - -import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; -import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; - -public class StreamsSmokeTest { - - /** - * args ::= kafka propFileName command disableAutoTerminate - * command := "run" | "process" - * - * @param args - */ - public static void main(final String[] args) throws IOException { - if (args.length < 2) { - System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); - System.exit(1); - } - - final String propFileName = args[0]; - final String command = args[1]; - final boolean disableAutoTerminate = args.length > 2; - - final Properties streamsProperties = Utils.loadProps(propFileName); - final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); - - if (kafka == null) { - System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - System.exit(1); - } - - if ("process".equals(command)) { - if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { - - System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + - StreamsConfig.EXACTLY_ONCE); - - System.exit(1); - } - } - - System.out.println("StreamsTest instance started (StreamsSmokeTest)"); - System.out.println("command=" + command); - System.out.println("props=" + streamsProperties); - System.out.println("disableAutoTerminate=" + disableAutoTerminate); - - switch (command) { - case "run": - // this starts the driver (data generation and result verification) - final int numKeys = 10; - final int maxRecordsPerKey = 500; - if (disableAutoTerminate) { - generatePerpetually(kafka, numKeys, maxRecordsPerKey); - } else { - // slow down data production to span 30 seconds so that system tests have time to - // do their bounces, etc. - final Map> allData = - generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); - SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); - } - break; - case "process": - // this starts the stream processing app - new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); - break; - default: - System.out.println("unknown command: " + command); - } - } - -} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java deleted file mode 100644 index ced1369af3855..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ /dev/null @@ -1,298 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.KafkaThread; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.Topology; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Grouped; -import org.apache.kafka.streams.kstream.KGroupedStream; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; -import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.streams.state.WindowStore; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; - -public class SmokeTestClient extends SmokeTestUtil { - - private final String name; - - private KafkaStreams streams; - private boolean uncaughtException = false; - private boolean started; - private volatile boolean closed; - - private static void addShutdownHook(final String name, final Runnable runnable) { - if (name != null) { - Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); - } else { - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } - } - - private static File tempDirectory() { - final String prefix = "kafka-"; - final File file; - try { - file = Files.createTempDirectory(prefix).toFile(); - } catch (final IOException ex) { - throw new RuntimeException("Failed to create a temp dir", ex); - } - file.deleteOnExit(); - - addShutdownHook("delete-temp-file-shutdown-hook", () -> { - try { - Utils.delete(file); - } catch (final IOException e) { - System.out.println("Error deleting " + file.getAbsolutePath()); - e.printStackTrace(System.out); - } - }); - - return file; - } - - public SmokeTestClient(final String name) { - this.name = name; - } - - public boolean started() { - return started; - } - - public boolean closed() { - return closed; - } - - public void start(final Properties streamsProperties) { - final Topology build = getTopology(); - streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); - - final CountDownLatch countDownLatch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); - if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { - started = true; - countDownLatch.countDown(); - } - - if (newState == KafkaStreams.State.NOT_RUNNING) { - closed = true; - } - }); - - streams.setUncaughtExceptionHandler((t, e) -> { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - System.out.println(name + ": FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - e.printStackTrace(System.out); - uncaughtException = true; - streams.close(Duration.ofSeconds(30)); - }); - - addShutdownHook("streams-shutdown-hook", this::close); - - streams.start(); - try { - if (!countDownLatch.await(1, TimeUnit.MINUTES)) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); - } - } catch (final InterruptedException e) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); - e.printStackTrace(System.out); - } - System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); - System.out.println(name + " started at " + Instant.now()); - } - - public void closeAsync() { - streams.close(Duration.ZERO); - } - - public void close() { - final boolean closed = streams.close(Duration.ofMinutes(1)); - - if (closed && !uncaughtException) { - System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); - } else if (closed) { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); - } else { - System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); - } - } - - private Properties getStreamsConfig(final Properties props) { - final Properties fullProps = new Properties(props); - fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); - fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); - fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); - fullProps.putAll(props); - return fullProps; - } - - public Topology getTopology() { - final StreamsBuilder builder = new StreamsBuilder(); - final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); - final KStream source = builder.stream("data", stringIntConsumed); - source.filterNot((k, v) -> k.equals("flush")) - .to("echo", Produced.with(stringSerde, intSerde)); - final KStream data = source.filter((key, value) -> value == null || value != END); - data.process(SmokeTestUtil.printProcessorSupplier("data", name)); - - // min - final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); - - final KTable, Integer> minAggregation = groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(1)).grace(Duration.ofMinutes(1))) - .aggregate( - () -> Integer.MAX_VALUE, - (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, - Materialized - .>as("uwin-min") - .withValueSerde(intSerde) - .withRetention(Duration.ofHours(25)) - ); - - streamify(minAggregation, "min-raw"); - - streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); - - minAggregation - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("min", Produced.with(stringSerde, intSerde)); - - final KTable, Integer> smallWindowSum = groupedData - .windowedBy(TimeWindows.of(Duration.ofSeconds(2)).advanceBy(Duration.ofSeconds(1)).grace(Duration.ofSeconds(30))) - .reduce((l, r) -> l + r); - - streamify(smallWindowSum, "sws-raw"); - streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); - - final KTable minTable = builder.table( - "min", - Consumed.with(stringSerde, intSerde), - Materialized.as("minStoreName")); - - minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); - - // max - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> Integer.MIN_VALUE, - (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, - Materialized.>as("uwin-max").withValueSerde(intSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("max", Produced.with(stringSerde, intSerde)); - - final KTable maxTable = builder.table( - "max", - Consumed.with(stringSerde, intSerde), - Materialized.as("maxStoreName")); - maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); - - // sum - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .aggregate( - () -> 0L, - (aggKey, value, aggregate) -> (long) value + aggregate, - Materialized.>as("win-sum").withValueSerde(longSerde)) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("sum", Produced.with(stringSerde, longSerde)); - - final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); - final KTable sumTable = builder.table("sum", stringLongConsumed); - sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); - - // cnt - groupedData - .windowedBy(TimeWindows.of(Duration.ofDays(2))) - .count(Materialized.as("uwin-cnt")) - .toStream(new Unwindow<>()) - .filterNot((k, v) -> k.equals("flush")) - .to("cnt", Produced.with(stringSerde, longSerde)); - - final KTable cntTable = builder.table( - "cnt", - Consumed.with(stringSerde, longSerde), - Materialized.as("cntStoreName")); - cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); - - // dif - maxTable - .join( - minTable, - (value1, value2) -> value1 - value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("dif", Produced.with(stringSerde, intSerde)); - - // avg - sumTable - .join( - cntTable, - (value1, value2) -> (double) value1 / (double) value2) - .toStream() - .filterNot((k, v) -> k.equals("flush")) - .to("avg", Produced.with(stringSerde, doubleSerde)); - - // test repartition - final Agg agg = new Agg(); - cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) - .aggregate(agg.init(), agg.adder(), agg.remover(), - Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) - .withKeySerde(Serdes.String()) - .withValueSerde(Serdes.Long())) - .toStream() - .to("tagg", Produced.with(stringSerde, longSerde)); - - return builder.build(); - } - - private static void streamify(final KTable, Integer> windowedTable, final String topic) { - windowedTable - .toStream() - .filterNot((k, v) -> k.key().equals("flush")) - .map((key, value) -> new KeyValue<>(key.toString(), value)) - .to(topic, Produced.with(stringSerde, intSerde)); - } -} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java deleted file mode 100644 index ac83cd95ebaaf..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ /dev/null @@ -1,622 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.utils.Utils; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; - -public class SmokeTestDriver extends SmokeTestUtil { - private static final String[] TOPICS = { - "data", - "echo", - "max", - "min", "min-suppressed", "min-raw", - "dif", - "sum", - "sws-raw", "sws-suppressed", - "cnt", - "avg", - "tagg" - }; - - private static final int MAX_RECORD_EMPTY_RETRIES = 30; - - private static class ValueList { - public final String key; - private final int[] values; - private int index; - - ValueList(final int min, final int max) { - key = min + "-" + max; - - values = new int[max - min + 1]; - for (int i = 0; i < values.length; i++) { - values[i] = min + i; - } - // We want to randomize the order of data to test not completely predictable processing order - // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) - // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window - shuffle(values, 10); - - index = 0; - } - - int next() { - return (index < values.length) ? values[index++] : -1; - } - } - - public static String[] topics() { - return Arrays.copyOf(TOPICS, TOPICS.length); - } - - static void generatePerpetually(final String kafka, - final int numKeys, - final int maxRecordsPerKey) { - final Properties producerProps = generatorProperties(kafka); - - int numRecordsProduced = 0; - - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - } - - final Random rand = new Random(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (true) { - final int index = rand.nextInt(numKeys); - final String key = data[index].key; - final int value = data[index].next(); - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record); - - numRecordsProduced++; - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(2); - } - } - } - - public static Map> generate(final String kafka, - final int numKeys, - final int maxRecordsPerKey, - final Duration timeToSpend) { - final Properties producerProps = generatorProperties(kafka); - - - int numRecordsProduced = 0; - - final Map> allData = new HashMap<>(); - final ValueList[] data = new ValueList[numKeys]; - for (int i = 0; i < numKeys; i++) { - data[i] = new ValueList(i, i + maxRecordsPerKey - 1); - allData.put(data[i].key, new HashSet<>()); - } - final Random rand = new Random(); - - int remaining = data.length; - - final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; - - List> needRetry = new ArrayList<>(); - - try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { - while (remaining > 0) { - final int index = rand.nextInt(remaining); - final String key = data[index].key; - final int value = data[index].next(); - - if (value < 0) { - remaining--; - data[index] = data[remaining]; - } else { - - final ProducerRecord record = - new ProducerRecord<>( - "data", - stringSerde.serializer().serialize("", key), - intSerde.serializer().serialize("", value) - ); - - producer.send(record, new TestCallback(record, needRetry)); - - numRecordsProduced++; - allData.get(key).add(value); - if (numRecordsProduced % 100 == 0) { - System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); - } - Utils.sleep(Math.max(recordPauseTime, 2)); - } - } - producer.flush(); - - int remainingRetries = 5; - while (!needRetry.isEmpty()) { - final List> needRetry2 = new ArrayList<>(); - for (final ProducerRecord record : needRetry) { - System.out.println("retry producing " + stringSerde.deserializer().deserialize("", record.key())); - producer.send(record, new TestCallback(record, needRetry2)); - } - producer.flush(); - needRetry = needRetry2; - - if (--remainingRetries == 0 && !needRetry.isEmpty()) { - System.err.println("Failed to produce all records after multiple retries"); - Exit.exit(1); - } - } - - // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out - // all suppressed records. - final List partitions = producer.partitionsFor("data"); - for (final PartitionInfo partition : partitions) { - producer.send(new ProducerRecord<>( - partition.topic(), - partition.partition(), - System.currentTimeMillis() + Duration.ofDays(2).toMillis(), - stringSerde.serializer().serialize("", "flush"), - intSerde.serializer().serialize("", 0) - )); - } - } - return Collections.unmodifiableMap(allData); - } - - private static Properties generatorProperties(final String kafka) { - final Properties producerProps = new Properties(); - producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); - return producerProps; - } - - private static class TestCallback implements Callback { - private final ProducerRecord originalRecord; - private final List> needRetry; - - TestCallback(final ProducerRecord originalRecord, - final List> needRetry) { - this.originalRecord = originalRecord; - this.needRetry = needRetry; - } - - @Override - public void onCompletion(final RecordMetadata metadata, final Exception exception) { - if (exception != null) { - if (exception instanceof TimeoutException) { - needRetry.add(originalRecord); - } else { - exception.printStackTrace(); - Exit.exit(1); - } - } - } - } - - private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { - final Random rand = new Random(); - for (int i = 0; i < data.length; i++) { - // we shuffle data within windowSize - final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; - - // swap - final int tmp = data[i]; - data[i] = data[j]; - data[j] = tmp; - } - } - - public static class NumberDeserializer implements Deserializer { - @Override - public Number deserialize(final String topic, final byte[] data) { - final Number value; - switch (topic) { - case "data": - case "echo": - case "min": - case "min-raw": - case "min-suppressed": - case "sws-raw": - case "sws-suppressed": - case "max": - case "dif": - value = intSerde.deserializer().deserialize(topic, data); - break; - case "sum": - case "cnt": - case "tagg": - value = longSerde.deserializer().deserialize(topic, data); - break; - case "avg": - value = doubleSerde.deserializer().deserialize(topic, data); - break; - default: - throw new RuntimeException("unknown topic: " + topic); - } - return value; - } - } - - public static VerificationResult verify(final String kafka, - final Map> inputs, - final int maxRecordsPerKey) { - final Properties props = new Properties(); - props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); - props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); - - final KafkaConsumer consumer = new KafkaConsumer<>(props); - final List partitions = getAllPartitions(consumer, TOPICS); - consumer.assign(partitions); - consumer.seekToBeginning(partitions); - - final int recordsGenerated = inputs.size() * maxRecordsPerKey; - int recordsProcessed = 0; - final Map processed = - Stream.of(TOPICS) - .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); - - final Map>>> events = new HashMap<>(); - - VerificationResult verificationResult = new VerificationResult(false, "no results yet"); - int retry = 0; - final long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { - final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); - if (records.isEmpty() && recordsProcessed >= recordsGenerated) { - verificationResult = verifyAll(inputs, events, false); - if (verificationResult.passed()) { - break; - } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { - System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); - break; - } else { - System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); - } - } else { - System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); - - retry = 0; - for (final ConsumerRecord record : records) { - final String key = record.key(); - - final String topic = record.topic(); - processed.get(topic).incrementAndGet(); - - if (topic.equals("echo")) { - recordsProcessed++; - if (recordsProcessed % 100 == 0) { - System.out.println("Echo records processed = " + recordsProcessed); - } - } - - events.computeIfAbsent(topic, t -> new HashMap<>()) - .computeIfAbsent(key, k -> new LinkedList<>()) - .add(record); - } - - System.out.println(processed); - } - } - consumer.close(); - final long finished = System.currentTimeMillis() - start; - System.out.println("Verification time=" + finished); - System.out.println("-------------------"); - System.out.println("Result Verification"); - System.out.println("-------------------"); - System.out.println("recordGenerated=" + recordsGenerated); - System.out.println("recordProcessed=" + recordsProcessed); - - if (recordsProcessed > recordsGenerated) { - System.out.println("PROCESSED-MORE-THAN-GENERATED"); - } else if (recordsProcessed < recordsGenerated) { - System.out.println("PROCESSED-LESS-THAN-GENERATED"); - } - - boolean success; - - final Map> received = - events.get("echo") - .entrySet() - .stream() - .map(entry -> mkEntry( - entry.getKey(), - entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) - ) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - success = inputs.equals(received); - - if (success) { - System.out.println("ALL-RECORDS-DELIVERED"); - } else { - int missedCount = 0; - for (final Map.Entry> entry : inputs.entrySet()) { - missedCount += received.get(entry.getKey()).size(); - } - System.out.println("missedRecords=" + missedCount); - } - - // give it one more try if it's not already passing. - if (!verificationResult.passed()) { - verificationResult = verifyAll(inputs, events, true); - } - success &= verificationResult.passed(); - - System.out.println(verificationResult.result()); - - System.out.println(success ? "SUCCESS" : "FAILURE"); - return verificationResult; - } - - public static class VerificationResult { - private final boolean passed; - private final String result; - - VerificationResult(final boolean passed, final String result) { - this.passed = passed; - this.result = result; - } - - public boolean passed() { - return passed; - } - - public String result() { - return result; - } - } - - private static VerificationResult verifyAll(final Map> inputs, - final Map>>> events, - final boolean printResults) { - final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - boolean pass; - try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { - pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); - pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); - pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { - final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); - return getMin(unwindowedKey); - }, printResults); - pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); - pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); - pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); - pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); - pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); - pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); - pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); - } - return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); - } - - private static boolean verify(final PrintStream resultStream, - final String topic, - final Map> inputData, - final Map>>> events, - final Function keyToExpectation, - final boolean printResults) { - final Map>> observedInputEvents = events.get("data"); - final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); - if (outputEvents.isEmpty()) { - resultStream.println(topic + " is empty"); - return false; - } else { - resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); - - if (outputEvents.size() != inputData.size()) { - resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", - outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); - return false; - } - for (final Map.Entry>> entry : outputEvents.entrySet()) { - final String key = entry.getKey(); - final Number expected = keyToExpectation.apply(key); - final Number actual = entry.getValue().getLast().value(); - if (!expected.equals(actual)) { - resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); - - if (printResults) { - resultStream.printf("\t inputEvents=%n%s%n\t" + - "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", - indent("\t\t", observedInputEvents.get(key)), - indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), - indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); - - if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) - resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); - } - - return false; - } - } - return true; - } - } - - - private static boolean verifySuppressed(final PrintStream resultStream, - @SuppressWarnings("SameParameterValue") final String topic, - final Map>>> events, - final boolean printResults) { - resultStream.println("verifying suppressed " + topic); - final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); - for (final Map.Entry>> entry : topicEvents.entrySet()) { - if (entry.getValue().size() != 1) { - final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); - final String key = entry.getKey(); - final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); - resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", - key, - indent("\t\t", entry.getValue())); - - if (printResults) - resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", - indent("\t\t", events.get(unsuppressedTopic).get(key)), - indent("\t\t", events.get("data").get(unwindowedKey))); - - return false; - } - } - return true; - } - - private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, - final Iterable> list) { - final StringBuilder stringBuilder = new StringBuilder(); - for (final ConsumerRecord record : list) { - stringBuilder.append(prefix).append(record).append('\n'); - } - return stringBuilder.toString(); - } - - private static Long getSum(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) * (max - min + 1L) / 2L; - } - - private static Double getAvg(final String key) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - return ((long) min + max) / 2.0; - } - - - private static boolean verifyTAgg(final PrintStream resultStream, - final Map> allData, - final Map>> taggEvents, - final boolean printResults) { - if (taggEvents == null) { - resultStream.println("tagg is missing"); - return false; - } else if (taggEvents.isEmpty()) { - resultStream.println("tagg is empty"); - return false; - } else { - resultStream.println("verifying tagg"); - - // generate expected answer - final Map expected = new HashMap<>(); - for (final String key : allData.keySet()) { - final int min = getMin(key).intValue(); - final int max = getMax(key).intValue(); - final String cnt = Long.toString(max - min + 1L); - - expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); - } - - // check the result - for (final Map.Entry>> entry : taggEvents.entrySet()) { - final String key = entry.getKey(); - Long expectedCount = expected.remove(key); - if (expectedCount == null) { - expectedCount = 0L; - } - - if (entry.getValue().getLast().value().longValue() != expectedCount) { - resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); - - if (printResults) - resultStream.println("\t taggEvents: " + entry.getValue()); - return false; - } - } - - } - return true; - } - - private static Number getMin(final String key) { - return Integer.parseInt(key.split("-")[0]); - } - - private static Number getMax(final String key) { - return Integer.parseInt(key.split("-")[1]); - } - - private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { - final List partitions = new ArrayList<>(); - - for (final String topic : topics) { - for (final PartitionInfo info : consumer.partitionsFor(topic)) { - partitions.add(new TopicPartition(info.topic(), info.partition())); - } - } - return partitions; - } - -} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java deleted file mode 100644 index e8ec04c23ca52..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -import java.time.Instant; - -public class SmokeTestUtil { - - final static int END = Integer.MAX_VALUE; - - static ProcessorSupplier printProcessorSupplier(final String topic) { - return printProcessorSupplier(topic, ""); - } - - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return new ProcessorSupplier() { - @Override - public Processor get() { - return new AbstractProcessor() { - private int numRecordsProcessed = 0; - private long smallestOffset = Long.MAX_VALUE; - private long largestOffset = Long.MIN_VALUE; - - @Override - public void init(final ProcessorContext context) { - super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); - System.out.flush(); - numRecordsProcessed = 0; - smallestOffset = Long.MAX_VALUE; - largestOffset = Long.MIN_VALUE; - } - - @Override - public void process(final Object key, final Object value) { - numRecordsProcessed++; - if (numRecordsProcessed % 100 == 0) { - System.out.printf("%s: %s%n", name, Instant.now()); - System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); - } - - if (smallestOffset > context().offset()) { - smallestOffset = context().offset(); - } - if (largestOffset < context().offset()) { - largestOffset = context().offset(); - } - } - - @Override - public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); - System.out.println("processed " + numRecordsProcessed + " records"); - final long processed; - if (largestOffset >= smallestOffset) { - processed = 1L + largestOffset - smallestOffset; - } else { - processed = 0L; - } - System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); - System.out.flush(); - } - }; - } - }; - } - - public static final class Unwindow implements KeyValueMapper, V, K> { - @Override - public K apply(final Windowed winKey, final V value) { - return winKey.key(); - } - } - - public static class Agg { - - KeyValueMapper> selector() { - return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); - } - - public Initializer init() { - return () -> 0L; - } - - Aggregator adder() { - return (aggKey, value, aggregate) -> aggregate + value; - } - - Aggregator remover() { - return (aggKey, value, aggregate) -> aggregate - value; - } - } - - public static Serde stringSerde = Serdes.String(); - - public static Serde intSerde = Serdes.Integer(); - - static Serde longSerde = Serdes.Long(); - - static Serde doubleSerde = Serdes.Double(); - - public static void sleep(final long duration) { - try { - Thread.sleep(duration); - } catch (final Exception ignore) { } - } - -} diff --git a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java deleted file mode 100644 index 07c7d5d82ebff..0000000000000 --- a/streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.tests; - -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.StreamsConfig; - -import java.io.IOException; -import java.time.Duration; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; - -import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; -import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; - -public class StreamsSmokeTest { - - /** - * args ::= kafka propFileName command disableAutoTerminate - * command := "run" | "process" - * - * @param args - */ - public static void main(final String[] args) throws IOException { - if (args.length < 2) { - System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); - System.exit(1); - } - - final String propFileName = args[0]; - final String command = args[1]; - final boolean disableAutoTerminate = args.length > 2; - - final Properties streamsProperties = Utils.loadProps(propFileName); - final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); - - if (kafka == null) { - System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); - System.exit(1); - } - - if ("process".equals(command)) { - if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee)) { - - System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + - StreamsConfig.EXACTLY_ONCE); - - System.exit(1); - } - } - - System.out.println("StreamsTest instance started (StreamsSmokeTest)"); - System.out.println("command=" + command); - System.out.println("props=" + streamsProperties); - System.out.println("disableAutoTerminate=" + disableAutoTerminate); - - switch (command) { - case "run": - // this starts the driver (data generation and result verification) - final int numKeys = 10; - final int maxRecordsPerKey = 500; - if (disableAutoTerminate) { - generatePerpetually(kafka, numKeys, maxRecordsPerKey); - } else { - // slow down data production to span 30 seconds so that system tests have time to - // do their bounces, etc. - final Map> allData = - generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); - SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); - } - break; - case "process": - // this starts the stream processing app - new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); - break; - default: - System.out.println("unknown command: " + command); - } - } - -} diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 354db13f8460a..72847f6328722 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -305,62 +305,23 @@ def start_node(self, node): class StreamsSmokeTestBaseService(StreamsTestBaseService): """Base class for Streams Smoke Test services providing some common settings and functionality""" - def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', num_threads = 3, replication_factor = 3): + def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', num_threads = 3): super(StreamsSmokeTestBaseService, self).__init__(test_context, kafka, "org.apache.kafka.streams.tests.StreamsSmokeTest", command) self.NUM_THREADS = num_threads self.PROCESSING_GUARANTEE = processing_guarantee - self.KAFKA_STREAMS_VERSION = "" - self.UPGRADE_FROM = None - self.REPLICATION_FACTOR = replication_factor - - def set_version(self, kafka_streams_version): - self.KAFKA_STREAMS_VERSION = kafka_streams_version - - def set_upgrade_from(self, upgrade_from): - self.UPGRADE_FROM = upgrade_from def prop_file(self): properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT, streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(), streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE, - streams_property.NUM_THREADS: self.NUM_THREADS, - "replication.factor": self.REPLICATION_FACTOR, - "num.standby.replicas": 2, - "buffered.records.per.partition": 100, - "commit.interval.ms": 1000, - "auto.offset.reset": "earliest", - "acks": "all"} - - if self.UPGRADE_FROM is not None: - properties['upgrade.from'] = self.UPGRADE_FROM + streams_property.NUM_THREADS: self.NUM_THREADS} cfg = KafkaConfig(**properties) return cfg.render() - def start_cmd(self, node): - args = self.args.copy() - args['config_file'] = self.CONFIG_FILE - args['stdout'] = self.STDOUT_FILE - args['stderr'] = self.STDERR_FILE - args['pidfile'] = self.PID_FILE - args['log4j'] = self.LOG4J_CONFIG_FILE - args['version'] = self.KAFKA_STREAMS_VERSION - args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) - - cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\";" \ - " INCLUDE_TEST_JARS=true UPGRADE_KAFKA_STREAMS_TEST_VERSION=%(version)s" \ - " %(kafka_run_class)s %(streams_class_name)s" \ - " %(config_file)s %(user_test_args1)s" \ - " & echo $! >&3 ) " \ - "1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args - - self.logger.info("Executing streams cmd: " + cmd) - - return cmd - class StreamsEosTestBaseService(StreamsTestBaseService): """Base class for Streams EOS Test services providing some common settings and functionality""" @@ -409,18 +370,17 @@ def start_cmd(self, node): " %(config_file)s %(user_test_args1)s %(disable_auto_terminate)s" \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args - self.logger.info("Executing streams cmd: " + cmd) - return cmd class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): - def __init__(self, test_context, kafka, processing_guarantee, num_threads = 3, replication_factor = 3): - super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, num_threads, replication_factor) + def __init__(self, test_context, kafka, processing_guarantee, num_threads = 3): + super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, num_threads) class StreamsEosTestDriverService(StreamsEosTestBaseService): def __init__(self, test_context, kafka): super(StreamsEosTestDriverService, self).__init__(test_context, kafka, "not-required", "run") + class StreamsEosTestJobRunnerService(StreamsEosTestBaseService): def __init__(self, test_context, kafka, processing_guarantee): super(StreamsEosTestJobRunnerService, self).__init__(test_context, kafka, processing_guarantee, "process") diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 37f177a82385f..0f0719a3ef8c9 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -37,9 +37,6 @@ # can be replaced with metadata_2_versions backward_compatible_metadata_2_versions = [str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)] metadata_3_or_higher_versions = [str(LATEST_2_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(DEV_VERSION)] -smoke_test_versions = [str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5)] - -dev_version = [str(DEV_VERSION)] """ After each release one should first check that the released version has been uploaded to @@ -192,8 +189,8 @@ def test_upgrade_downgrade_brokers(self, from_version, to_version): processor.stop() processor.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % processor.STDOUT_FILE, allow_fail=False) - @matrix(from_version=smoke_test_versions, to_version=dev_version) - def test_app_upgrade(self, from_version, to_version): + @matrix(from_version=metadata_2_versions, to_version=metadata_2_versions) + def test_simple_upgrade_downgrade(self, from_version, to_version): """ Starts 3 KafkaStreams instances with , and upgrades one-by-one to """ @@ -204,29 +201,14 @@ def test_app_upgrade(self, from_version, to_version): self.zk = ZookeeperService(self.test_context, num_nodes=1) self.zk.start() - self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics={ - 'echo' : { 'partitions': 5, 'replication-factor': 1 }, - 'data' : { 'partitions': 5, 'replication-factor': 1 }, - 'min' : { 'partitions': 5, 'replication-factor': 1 }, - 'min-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, - 'min-raw' : { 'partitions': 5, 'replication-factor': 1 }, - 'max' : { 'partitions': 5, 'replication-factor': 1 }, - 'sum' : { 'partitions': 5, 'replication-factor': 1 }, - 'sws-raw' : { 'partitions': 5, 'replication-factor': 1 }, - 'sws-suppressed' : { 'partitions': 5, 'replication-factor': 1 }, - 'dif' : { 'partitions': 5, 'replication-factor': 1 }, - 'cnt' : { 'partitions': 5, 'replication-factor': 1 }, - 'avg' : { 'partitions': 5, 'replication-factor': 1 }, - 'wcnt' : { 'partitions': 5, 'replication-factor': 1 }, - 'tagg' : { 'partitions': 5, 'replication-factor': 1 } - }) + self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics=self.topics) self.kafka.start() self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.driver.disable_auto_terminate() - self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) - self.processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) - self.processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee = "at_least_once", replication_factor = 1) + self.processor1 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor2 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) + self.processor3 = StreamsUpgradeTestJobRunnerService(self.test_context, self.kafka) self.driver.start() self.start_all_nodes_with(from_version) @@ -246,17 +228,14 @@ def test_app_upgrade(self, from_version, to_version): # shutdown self.driver.stop() - # Ideally, we would actually verify the expected results. - # See KAFKA-10202 - random.shuffle(self.processors) for p in self.processors: node = p.node with node.account.monitor_log(p.STDOUT_FILE) as monitor: p.stop() - monitor.wait_until("SMOKE-TEST-CLIENT-CLOSED", + monitor.wait_until("UPGRADE-TEST-CLIENT-CLOSED", timeout_sec=60, - err_msg="Never saw output 'SMOKE-TEST-CLIENT-CLOSED' on " + str(node.account)) + err_msg="Never saw output 'UPGRADE-TEST-CLIENT-CLOSED' on" + str(node.account)) @matrix(from_version=metadata_1_versions, to_version=backward_compatible_metadata_2_versions) @matrix(from_version=metadata_1_versions, to_version=metadata_3_or_higher_versions) @@ -370,42 +349,56 @@ def get_version_string(self, version): def start_all_nodes_with(self, version): kafka_version_str = self.get_version_string(version) + # start first with self.prepare_for(self.processor1, version) - self.prepare_for(self.processor2, version) - self.prepare_for(self.processor3, version) - - self.processor1.start() - self.processor2.start() - self.processor3.start() - - # double-check the version - self.wait_for_verification(self.processor1, kafka_version_str, self.processor1.LOG_FILE) - self.wait_for_verification(self.processor2, kafka_version_str, self.processor2.LOG_FILE) - self.wait_for_verification(self.processor3, kafka_version_str, self.processor3.LOG_FILE) - - # wait for the members to join - self.wait_for_verification(self.processor1, "SMOKE-TEST-CLIENT-STARTED", self.processor1.STDOUT_FILE) - self.wait_for_verification(self.processor2, "SMOKE-TEST-CLIENT-STARTED", self.processor2.STDOUT_FILE) - self.wait_for_verification(self.processor3, "SMOKE-TEST-CLIENT-STARTED", self.processor3.STDOUT_FILE) - - # make sure they've processed something - self.wait_for_verification(self.processor1, self.processed_msg, self.processor1.STDOUT_FILE) - self.wait_for_verification(self.processor2, self.processed_msg, self.processor2.STDOUT_FILE) - self.wait_for_verification(self.processor3, self.processed_msg, self.processor3.STDOUT_FILE) + node1 = self.processor1.node + with node1.account.monitor_log(self.processor1.STDOUT_FILE) as monitor: + with node1.account.monitor_log(self.processor1.LOG_FILE) as log_monitor: + self.processor1.start() + log_monitor.wait_until(kafka_version_str, + timeout_sec=60, + err_msg="Could not detect Kafka Streams version " + version + " " + str(node1.account)) + monitor.wait_until(self.processed_msg, + timeout_sec=60, + err_msg="Never saw output '%s' on " % self.processed_msg + str(node1.account)) - def wait_for_verification(self, processor, message, file, num_lines=1): - wait_until(lambda: self.verify_from_file(processor, message, file) >= num_lines, - timeout_sec=60, - err_msg="Did expect to read '%s' from %s" % (message, processor.node.account)) + # start second with + self.prepare_for(self.processor2, version) + node2 = self.processor2.node + with node1.account.monitor_log(self.processor1.STDOUT_FILE) as first_monitor: + with node2.account.monitor_log(self.processor2.STDOUT_FILE) as second_monitor: + with node2.account.monitor_log(self.processor2.LOG_FILE) as log_monitor: + self.processor2.start() + log_monitor.wait_until(kafka_version_str, + timeout_sec=60, + err_msg="Could not detect Kafka Streams version " + version + " on " + str(node2.account)) + first_monitor.wait_until(self.processed_msg, + timeout_sec=60, + err_msg="Never saw output '%s' on " % self.processed_msg + str(node1.account)) + second_monitor.wait_until(self.processed_msg, + timeout_sec=60, + err_msg="Never saw output '%s' on " % self.processed_msg + str(node2.account)) - @staticmethod - def verify_from_file(processor, message, file): - result = processor.node.account.ssh_output("grep -E '%s' %s | wc -l" % (message, file), allow_fail=False) - try: - return int(result) - except ValueError: - self.logger.warn("Command failed with ValueError: " + result) - return 0 + # start third with + self.prepare_for(self.processor3, version) + node3 = self.processor3.node + with node1.account.monitor_log(self.processor1.STDOUT_FILE) as first_monitor: + with node2.account.monitor_log(self.processor2.STDOUT_FILE) as second_monitor: + with node3.account.monitor_log(self.processor3.STDOUT_FILE) as third_monitor: + with node3.account.monitor_log(self.processor3.LOG_FILE) as log_monitor: + self.processor3.start() + log_monitor.wait_until(kafka_version_str, + timeout_sec=60, + err_msg="Could not detect Kafka Streams version " + version + " on " + str(node3.account)) + first_monitor.wait_until(self.processed_msg, + timeout_sec=60, + err_msg="Never saw output '%s' on " % self.processed_msg + str(node1.account)) + second_monitor.wait_until(self.processed_msg, + timeout_sec=60, + err_msg="Never saw output '%s' on " % self.processed_msg + str(node2.account)) + third_monitor.wait_until(self.processed_msg, + timeout_sec=60, + err_msg="Never saw output '%s' on " % self.processed_msg + str(node3.account)) @staticmethod def prepare_for(processor, version): From f5cc0b72f12cb3ee2efb207913240539b1ec1d61 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Fri, 26 Jun 2020 18:56:42 -0500 Subject: [PATCH 23/23] style --- .../test/java/org/apache/kafka/common/utils/UtilsTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 47eefe0d9a883..3cc0043b9629a 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -591,7 +591,9 @@ private static class TestCloseable implements Closeable { @Override public void close() throws IOException { closed = true; - if (closeException != null) { throw closeException; } + if (closeException != null) { + throw closeException; + } } static TestCloseable[] createCloseables(boolean... exceptionOnClose) {