aliehsaeedii commented on code in PR #21666:
URL: https://github.com/apache/kafka/pull/21666#discussion_r2900321698


##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/HeadersStoreUpgradeIntegrationTest.java:
##########
@@ -233,6 +234,128 @@ public void 
shouldProxyTimestampedKeyValueStoreToTimestampedKeyValueStoreWithHea
         kafkaStreams.close();
     }
 
+    @Test
+    public void 
shouldMigrateInMemoryPlainKeyValueStoreToTimestampedKeyValueStoreWithHeadersUsingPapi()
 throws Exception {
+        
shouldMigratePlainKeyValueStoreToTimestampedKeyValueStoreWithHeadersUsingPapi(false);
+    }
+
+    @Test
+    public void 
shouldMigratePersistentPlainKeyValueStoreToTimestampedKeyValueStoreWithHeadersUsingPapi()
 throws Exception {
+        
shouldMigratePlainKeyValueStoreToTimestampedKeyValueStoreWithHeadersUsingPapi(true);
+    }
+
+    private void 
shouldMigratePlainKeyValueStoreToTimestampedKeyValueStoreWithHeadersUsingPapi(final
 boolean persistentStore) throws Exception {
+        final StreamsBuilder streamsBuilderForOldStore = new StreamsBuilder();
+
+        streamsBuilderForOldStore.addStateStore(
+                Stores.keyValueStoreBuilder(
+                    persistentStore ? 
Stores.persistentKeyValueStore(STORE_NAME) : 
Stores.inMemoryKeyValueStore(STORE_NAME),
+                    Serdes.String(),
+                    Serdes.String()))
+            .stream(inputStream, Consumed.with(Serdes.String(), 
Serdes.String()))
+            .process(KeyValueProcessor::new, STORE_NAME);
+
+        final Properties props = props();
+        kafkaStreams = new KafkaStreams(streamsBuilderForOldStore.build(), 
props);
+        kafkaStreams.start();
+
+        processKeyValueAndVerifyValue("key1", "value1");
+        processKeyValueAndVerifyValue("key2", "value2");
+        processKeyValueAndVerifyValue("key3", "value3");
+
+        kafkaStreams.close();
+        kafkaStreams = null;
+
+        final StreamsBuilder streamsBuilderForNewStore = new StreamsBuilder();
+
+        streamsBuilderForNewStore.addStateStore(
+                Stores.timestampedKeyValueStoreBuilderWithHeaders(
+                    persistentStore ? 
Stores.persistentTimestampedKeyValueStoreWithHeaders(STORE_NAME) : 
Stores.inMemoryKeyValueStore(STORE_NAME),
+                    Serdes.String(),
+                    Serdes.String()))
+            .stream(inputStream, Consumed.with(Serdes.String(), 
Serdes.String()))
+            .process(TimestampedKeyValueWithHeadersProcessor::new, STORE_NAME);
+
+        kafkaStreams = new KafkaStreams(streamsBuilderForNewStore.build(), 
props);
+        kafkaStreams.start();
+
+        if (persistentStore) {
+            // Verify legacy data can be read with empty headers and timestamp 
= -1
+            verifyLegacyValuesWithEmptyHeaders("key1", "value1", -1);
+            verifyLegacyValuesWithEmptyHeaders("key2", "value2", -1);
+            verifyLegacyValuesWithEmptyHeaders("key3", "value3", -1);
+        } else {

Review Comment:
   We should either accept that retrived records from changelog have record's 
ts instead of deafult ts (`-1`), or  change the `RAW_TO_WITH_HEADERS_INSTANCE`: 
https://github.com/apache/kafka/blob/03e6aa4398869ae20dc58976fa6c4215f9a24bc7/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java#L56



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to