You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ab...@apache.org on 2021/03/19 23:25:56 UTC

[kafka] branch 2.6 updated: KAFKA-12508: Emit records with same value and same timestamp (#10360)

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

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


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

commit 0ced0d227e66ebcfc1d03a611f6498591ef50379
Author: Bruno Cadonna <br...@confluent.io>
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 <ab...@apache.org>
---
 .../internals/ValueAndTimestampSerializer.java     |  35 +++---
 .../integration/EmitOnChangeIntegrationTest.java   | 130 +++++++++++++++++++++
 2 files changed, 149 insertions(+), 16 deletions(-)

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 58c6159..fa8aef4 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
@@ -46,6 +46,10 @@ public class ValueAndTimestampSerializer<V> implements WrappingNullableSerialize
         return true;
     }
 
+    private static boolean timeIsDecreasing(final byte[] oldRecord, final byte[] newRecord) {
+        return extractTimestamp(newRecord) <= extractTimestamp(oldRecord);
+    }
+
     private static long extractTimestamp(final byte[] bytes) {
         final byte[] timestampBytes = new byte[Long.BYTES];
         for (int i = 0; i < Long.BYTES; i++) {
@@ -55,31 +59,30 @@ public class ValueAndTimestampSerializer<V> implements WrappingNullableSerialize
     }
 
     /**
-     * @param left  the serialized byte array of the old record in state store
-     * @param right the serialized byte array of the new record being processed
+     * @param oldRecord  the serialized byte array of the old record in state store
+     * @param newRecord the serialized byte array of the new record being processed
      * @return true if the two serialized values are the same (excluding timestamp) or
      *              if the timestamp of right is less than left (indicating out of order record)
      *         false otherwise
      */
-    public static boolean compareValuesAndCheckForIncreasingTimestamp(final byte[] left, final byte[] right) {
-        if (left == right) {
+    public static boolean compareValuesAndCheckForIncreasingTimestamp(final byte[] oldRecord, final byte[] newRecord) {
+        if (oldRecord == newRecord) {
+            // same reference, so they are trivially the same (might both be null)
             return true;
-        }
-        if (left == null || right == null) {
+        } else if (oldRecord == null || newRecord == null) {
+            // only one is null, so they cannot be the same
             return false;
-        }
-
-        final int length = left.length;
-        if (right.length != length) {
+        } else if (newRecord.length != oldRecord.length) {
+            // they are different length, so they cannot be the same
             return false;
-        }
-
-        final long leftTimestamp = extractTimestamp(left);
-        final long rightTimestamp = extractTimestamp(right);
-        if (rightTimestamp < leftTimestamp) {
+        } else if (timeIsDecreasing(oldRecord, newRecord)) {
+            // the record time represents the beginning of the validity interval, so if the time
+            // moves backwards, we need to do the update regardless of whether the value has changed
             return false;
+        } else {
+            // all other checks have fallen through, so we actually compare the binary data of the two values
+            return skipTimestampAndCompareValues(oldRecord, newRecord);
         }
-        return skipTimestampAndCompareValues(left, right);
     }
 
     @Override
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..9ce8f14
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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.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, 2),
+                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)) {
+            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")
+                )
+            );
+        }
+    }
+}