This is an automated email from the ASF dual-hosted git repository.

ableegoldman pushed a commit to branch 2.8
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.8 by this push:
     new 28a692f  KAFKA-12508: Emit records with same value and same timestamp 
(#10360)
28a692f is described below

commit 28a692f3fb5a6b7c84b2282486f7dbc3e48616d7
Author: Bruno Cadonna <[email protected]>
AuthorDate: Fri Mar 19 23:51:19 2021 +0100

    KAFKA-12508: Emit records with same value and same timestamp (#10360)
    
    Emit on change introduced in Streams with KIP-557 might lead to
    data loss if a record is put into a source KTable and emitted
    downstream and then a failure happens before the offset could be
    committed. After Streams rereads the record, it would find a record
    with the same key, value and timestamp in the KTable (i.e. the same
    record that was put into the KTable before the failure) and not
    forward it downstreams. Hence, the record would never be processed
    downstream of the KTable which breaks at-least-once and exactly-once
    processing guarantees.
    
    Reviewers: Anna Sophie Blee-Goldman <[email protected]>
---
 .../internals/ValueAndTimestampSerializer.java     |   2 +-
 .../integration/EmitOnChangeIntegrationTest.java   | 132 +++++++++++++++++++++
 2 files changed, 133 insertions(+), 1 deletion(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueAndTimestampSerializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueAndTimestampSerializer.java
index 8f01dee..3c27ab6 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueAndTimestampSerializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueAndTimestampSerializer.java
@@ -105,7 +105,7 @@ public class ValueAndTimestampSerializer<V> implements 
WrappingNullableSerialize
     }
 
     private static boolean timeIsDecreasing(final byte[] oldRecord, final 
byte[] newRecord) {
-        return extractTimestamp(newRecord) < extractTimestamp(oldRecord);
+        return extractTimestamp(newRecord) <= extractTimestamp(oldRecord);
     }
 
     private static long extractTimestamp(final byte[] bytes) {
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java
new file mode 100644
index 0000000..79454ec
--- /dev/null
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+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.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+
+@Category(IntegrationTest.class)
+public class EmitOnChangeIntegrationTest {
+
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private static String inputTopic;
+    private static String outputTopic;
+    private static String appId = "";
+
+    @Before
+    public void setup() {
+        final String testId = safeUniqueTestName(getClass(), testName);
+        appId = "appId_" + testId;
+        inputTopic = "input" + testId;
+        outputTopic = "output" + testId;
+        IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic, 
outputTopic);
+    }
+
+    @Test
+    public void shouldEmitSameRecordAfterFailover() throws Exception {
+        final Properties properties  = mkObjectProperties(
+            mkMap(
+                mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+                mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+                mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath()),
+                mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1),
+                mkEntry(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0),
+                mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 300000),
+                mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.IntegerSerde.class),
+                mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class)
+            )
+        );
+
+        final AtomicBoolean shouldThrow = new AtomicBoolean(true);
+        final StreamsBuilder builder = new StreamsBuilder();
+        builder.table(inputTopic, Materialized.as("test-store"))
+            .toStream()
+            .map((key, value) -> {
+                if (shouldThrow.compareAndSet(true, false)) {
+                    throw new IllegalStateException("Kaboom");
+                } else {
+                    return new KeyValue<>(key, value);
+                }
+            })
+            .to(outputTopic);
+
+        try (final KafkaStreams kafkaStreams = new 
KafkaStreams(builder.build(), properties)) {
+            kafkaStreams.setUncaughtExceptionHandler(exception -> 
StreamThreadExceptionResponse.REPLACE_THREAD);
+            
StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams);
+
+            IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
+                inputTopic,
+                Arrays.asList(
+                    new KeyValue<>(1, "A"),
+                    new KeyValue<>(1, "B")
+                ),
+                TestUtils.producerConfig(
+                    CLUSTER.bootstrapServers(),
+                    IntegerSerializer.class,
+                    StringSerializer.class,
+                    new Properties()),
+                0L);
+
+            IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived(
+                TestUtils.consumerConfig(
+                    CLUSTER.bootstrapServers(),
+                    IntegerDeserializer.class,
+                    StringDeserializer.class
+                ),
+                outputTopic,
+                Arrays.asList(
+                    new KeyValue<>(1, "A"),
+                    new KeyValue<>(1, "B")
+                )
+            );
+        }
+    }
+}

Reply via email to