guozhangwang commented on a change in pull request #8905:
URL: https://github.com/apache/kafka/pull/8905#discussion_r446343541



##########
File path: tests/kafkatest/tests/streams/streams_upgrade_test.py
##########
@@ -189,8 +192,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=metadata_2_versions, to_version=metadata_2_versions)
-    def test_simple_upgrade_downgrade(self, from_version, to_version):
+    @matrix(from_version=smoke_test_versions, to_version=dev_version)

Review comment:
       +1, I think this is a great find.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java
##########
@@ -361,26 +366,20 @@ private void restoreBatch(final 
Collection<ConsumerRecord<byte[], byte[]>> batch
                             contextualRecord.recordContext()
                         )
                     );
-                } else if 
(V_2_CHANGELOG_HEADERS.lastHeader("v").equals(record.headers().lastHeader("v")))
 {
-                    // in this case, the changelog value is a serialized 
BufferValue
+                } else if (Arrays.equals(versionHeader.value(), 
V_2_CHANGELOG_HEADER_VALUE)) {
+
+                    final DeserializationResult deserializationResult = 
duckTypeV2(record, key);

Review comment:
       Could you clarify which comment are you referring to? I did not see any 
comments for the "restoreBatch" method..

##########
File path: 
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 {

Review comment:
       I'm assuming 22..25 client / drive code are all copy-pastes here so I 
skipped reviewing them. LMK if they aren't.

##########
File path: 
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<byte[], 
byte[]> 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;
+    }
+
+    private static DeserializationResult deserializeV2(final 
ConsumerRecord<byte[], byte[]> record,

Review comment:
       Some docs, either here or directly inside 
`InMemoryTimeOrderedKeyValueBuffer.java` explaining the format difference would 
help a lot. You can see some examples like `object GroupMetadataManager`

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java
##########
@@ -258,34 +263,43 @@ 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();
         ((RecordCollector.Supplier) context).recordCollector().send(
-                changelogTopic,
-                key,
-                buffer.array(),
-                V_2_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,

Review comment:
       I'm just thinking, maybe we should encode headers to tombstones too in 
case in the future we changed the semantics of tombstones?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to