UladzislauBlok commented on code in PR #21580:
URL: https://github.com/apache/kafka/pull/21580#discussion_r2869315709


##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java:
##########
@@ -270,38 +378,77 @@ public void testNotSendingOldValue() {
                 );
             final MockApiProcessor<String, Integer, Void, Void> proc1 = 
supplier.theCapturedProcessor();
 
-            inputTopic1.pipeInput("A", "01", 10L);
-            inputTopic1.pipeInput("B", "01", 20L);
-            inputTopic1.pipeInput("C", "01", 15L);
-            proc1.checkAndClearProcessResult(
-                new KeyValueTimestamp<>("A", new Change<>("01", null), 10),
-                new KeyValueTimestamp<>("B", new Change<>("01", null), 20),
-                new KeyValueTimestamp<>("C", new Change<>("01", null), 15)
-            );
+            final Headers headers = makeHeaders("test", "header");
+            inputTopic1.pipeInput(new TestRecord<>("A", "01", headers, 10L));
+            inputTopic1.pipeInput(new TestRecord<>("B", "01", headers, 20L));
+            inputTopic1.pipeInput(new TestRecord<>("C", "01", headers, 15L));
 
-            inputTopic1.pipeInput("A", "02", 8L);
-            inputTopic1.pipeInput("B", "02", 22L);
-            proc1.checkAndClearProcessResult(
-                new KeyValueTimestamp<>("A", new Change<>("02", null), 8),
-                new KeyValueTimestamp<>("B", new Change<>("02", null), 22)
-            );
+            if (storeFormat.equals("default")) {
+                proc1.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>("A", new Change<>("01", null), 10),
+                    new KeyValueTimestamp<>("B", new Change<>("01", null), 20),
+                    new KeyValueTimestamp<>("C", new Change<>("01", null), 15)
+                );
+            } else if (storeFormat.equals("headers")) {

Review Comment:
   We've reused `default` and `headers` many times. Can we use some constant? 
e.g., psf field or enum



##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java:
##########
@@ -217,38 +292,71 @@ public void testValueGetter() {
             final KTableValueGetter<String, String> getter1 = 
getterSupplier1.get();
             
getter1.init(driver.setCurrentNodeForProcessorContext(table1.name));
 
-            inputTopic1.pipeInput("A", "01", 10L);
-            inputTopic1.pipeInput("B", "01", 20L);
-            inputTopic1.pipeInput("C", "01", 15L);
-
-            assertEquals(ValueAndTimestamp.make("01", 10L), getter1.get("A"));
-            assertEquals(ValueAndTimestamp.make("01", 20L), getter1.get("B"));
-            assertEquals(ValueAndTimestamp.make("01", 15L), getter1.get("C"));
-
-            inputTopic1.pipeInput("A", "02", 30L);
-            inputTopic1.pipeInput("B", "02", 5L);
-
-            assertEquals(ValueAndTimestamp.make("02", 30L), getter1.get("A"));
-            assertEquals(ValueAndTimestamp.make("02", 5L), getter1.get("B"));
-            assertEquals(ValueAndTimestamp.make("01", 15L), getter1.get("C"));
-
-            inputTopic1.pipeInput("A", "03", 29L);
-
-            assertEquals(ValueAndTimestamp.make("03", 29L), getter1.get("A"));
-            assertEquals(ValueAndTimestamp.make("02", 5L), getter1.get("B"));
-            assertEquals(ValueAndTimestamp.make("01", 15L), getter1.get("C"));
-
-            inputTopic1.pipeInput("A", null, 50L);
-            inputTopic1.pipeInput("B", null, 3L);
-
-            assertNull(getter1.get("A"));
-            assertNull(getter1.get("B"));
-            assertEquals(ValueAndTimestamp.make("01", 15L), getter1.get("C"));
+            // Send records with unique headers for each key
+            final Headers headersA = makeHeaders("key", "A");
+            final Headers headersB = makeHeaders("key", "B");
+            final Headers headersC = makeHeaders("key", "C");
+
+            inputTopic1.pipeInput(new TestRecord<>("A", "01", headersA, 10L));
+            inputTopic1.pipeInput(new TestRecord<>("B", "01", headersB, 20L));
+            inputTopic1.pipeInput(new TestRecord<>("C", "01", headersC, 15L));
+
+            if (storeFormat.equals("defaults")) {
+                assertFalse(getter1.supportsHeaders(), "Getter should not 
support headers with 'default' format");
+                assertEquals(ValueAndTimestamp.make("01", 10L), 
getter1.get("A"));
+                assertEquals(ValueAndTimestamp.make("01", 20L), 
getter1.get("B"));
+                assertEquals(ValueAndTimestamp.make("01", 15L), 
getter1.get("C"));
+            } else if (storeFormat.equals("headers")) {
+                assertTrue(getter1.supportsHeaders(), "Getter should support 
headers with 'headers' format");
+                assertEquals(ValueTimestampHeaders.make("01", 10L, headersA), 
getter1.getWithHeaders("A"));
+                assertEquals(ValueTimestampHeaders.make("01", 20L, headersB), 
getter1.getWithHeaders("B"));
+                assertEquals(ValueTimestampHeaders.make("01", 15L, headersC), 
getter1.getWithHeaders("C"));
+            }
+
+            inputTopic1.pipeInput(new TestRecord<>("A", "02", headersA, 30L));
+            inputTopic1.pipeInput(new TestRecord<>("B", "02", headersB, 5L));

Review Comment:
   Why we send data out-of-order? Key `B` : 20L -> 5L



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerWithHeaders.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.ProcessorNode;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+import org.apache.kafka.streams.state.internals.CacheFlushListener;
+
+class TimestampedCacheFlushListenerWithHeaders<KOut, VOut> implements 
CacheFlushListener<KOut, ValueTimestampHeaders<VOut>> {
+
+    private final InternalProcessorContext<KOut, Change<VOut>> context;
+
+    @SuppressWarnings("rawtypes")
+    private final ProcessorNode myNode;
+
+    TimestampedCacheFlushListenerWithHeaders(final ProcessorContext<KOut, 
Change<VOut>> context) {
+        this.context = (InternalProcessorContext<KOut, Change<VOut>>) context;
+        myNode = this.context.currentNode();
+    }
+
+    @Override
+    public void apply(final Record<KOut, Change<ValueTimestampHeaders<VOut>>> 
record) {
+        @SuppressWarnings("rawtypes") final ProcessorNode prev = 
context.currentNode();
+        context.setCurrentNode(myNode);
+        try {
+            final VOut newValue = record.value().newValue != null ? 
record.value().newValue.value() : null;
+            final VOut oldValue = record.value().oldValue != null ? 
record.value().oldValue.value() : null;
+            final long timestamp = record.value().newValue != null ? 
record.value().newValue.timestamp() : record.timestamp();
+
+            context.forward(
+                record
+                    .withValue(new Change<>(newValue, oldValue, 
record.value().isLatest))
+                    .withTimestamp(timestamp)

Review Comment:
   I think headers are here:
   Line 54: .withHeaders(headers)



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java:
##########
@@ -34,12 +35,32 @@ default ValueAndTimestamp<V> get(final K key, final long 
asOfTimestamp) {
         throw new UnsupportedOperationException("get(key, timestamp) is only 
supported for versioned stores");
     }
 
+    /**
+     * Returns the latest record with value, timestamp, and headers. This 
method may only be
+     * called if {@link #supportsHeaders()} is true.
+     *
+     * @param key the key to get
+     * @return the value with timestamp and headers, or null if not found
+     */
+    default ValueTimestampHeaders<V> getWithHeaders(final K key) {

Review Comment:
   This is from the KIP:
   
   The goal of this KIP is therefore:
   - Provide a simple, explicit, and backwards-compatible way for DSL users to 
opt into headers-aware state stores, reusing the stores and upgrade paths 
defined in KIP-1271,
   - while keeping headers semantics of DSL operators out of scope for now 
(headers should be considered “empty” from a DSL semantics point of view in 
this KIP).
   
   I don't this this method is required for now, but from my perspective it's 
okay to introduce it
   This way we can fully implement state stores side of DSL, and next KIP can 
concentrate on DSL headers aware operations only



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueStoreWrapper.java:
##########
@@ -45,70 +53,80 @@ public class KeyValueStoreWrapper<K, V> implements 
StateStore {
     public static final long PUT_RETURN_CODE_IS_LATEST
         = VersionedKeyValueStore.PUT_RETURN_CODE_VALID_TO_UNDEFINED;
 
-    private TimestampedKeyValueStore<K, V> timestampedStore = null;
+    private TimestampedKeyValueStoreWithHeaders<K, V> headersStore = null;
     private VersionedKeyValueStore<K, V> versionedStore = null;

Review Comment:
   Why we leave non-headers versioned store?
   As I understand we have `VersionedKeyValueStoreWithHeaders`



##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java:
##########
@@ -325,33 +472,139 @@ public void testSendingOldValue() {
                 );
             final MockApiProcessor<String, Integer, Void, Void> proc1 = 
supplier.theCapturedProcessor();
 
-            inputTopic1.pipeInput("A", "01", 10L);
-            inputTopic1.pipeInput("B", "01", 20L);
-            inputTopic1.pipeInput("C", "01", 15L);
-            proc1.checkAndClearProcessResult(
-                new KeyValueTimestamp<>("A", new Change<>("01", null), 10),
-                new KeyValueTimestamp<>("B", new Change<>("01", null), 20),
-                new KeyValueTimestamp<>("C", new Change<>("01", null), 15)
-            );
+            final Headers headers = makeHeaders("test", "header");
+            inputTopic1.pipeInput(new TestRecord<>("A", "01", headers, 10L));
+            inputTopic1.pipeInput(new TestRecord<>("B", "01", headers, 20L));
+            inputTopic1.pipeInput(new TestRecord<>("C", "01", headers, 15L));
 
-            inputTopic1.pipeInput("A", "02", 8L);
-            inputTopic1.pipeInput("B", "02", 22L);
-            proc1.checkAndClearProcessResult(
-                new KeyValueTimestamp<>("A", new Change<>("02", "01"), 8),
-                new KeyValueTimestamp<>("B", new Change<>("02", "01"), 22)
-            );
+            if (storeFormat.equals("default")) {
+                proc1.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>("A", new Change<>("01", null), 10),
+                    new KeyValueTimestamp<>("B", new Change<>("01", null), 20),
+                    new KeyValueTimestamp<>("C", new Change<>("01", null), 15)
+                );
+            } else if (storeFormat.equals("headers")) {
+                proc1.checkAndClearProcessResultWithHeaders(
+                    new KeyValueTimestampHeaders<>("A", new Change<>("01", 
null), 10, headers),
+                    new KeyValueTimestampHeaders<>("B", new Change<>("01", 
null), 20, headers),
+                    new KeyValueTimestampHeaders<>("C", new Change<>("01", 
null), 15, headers)
+                );
+            }
 
-            inputTopic1.pipeInput("A", "03", 12L);
-            proc1.checkAndClearProcessResult(
-                new KeyValueTimestamp<>("A", new Change<>("03", "02"), 12)
-            );
+            inputTopic1.pipeInput(new TestRecord<>("A", "02", headers, 8L));
+            inputTopic1.pipeInput(new TestRecord<>("B", "02", headers, 22L));
+
+            if (storeFormat.equals("default")) {
+                proc1.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>("A", new Change<>("02", "01"), 8),
+                    new KeyValueTimestamp<>("B", new Change<>("02", "01"), 22)
+                );
+            } else if (storeFormat.equals("headers")) {
+                proc1.checkAndClearProcessResultWithHeaders(
+                    new KeyValueTimestampHeaders<>("A", new Change<>("02", 
"01"), 8, headers),
+                    new KeyValueTimestampHeaders<>("B", new Change<>("02", 
"01"), 22, headers)
+                );
+            }
+
+            inputTopic1.pipeInput(new TestRecord<>("A", "03", headers, 12L));
+
+            if (storeFormat.equals("default")) {
+                proc1.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>("A", new Change<>("03", "02"), 12)
+                );
+            } else if (storeFormat.equals("headers")) {
+                proc1.checkAndClearProcessResultWithHeaders(
+                    new KeyValueTimestampHeaders<>("A", new Change<>("03", 
"02"), 12, headers)
+                );
+            }
+
+            inputTopic1.pipeInput(new TestRecord<>("A", null, headers, 15L));
+            inputTopic1.pipeInput(new TestRecord<>("B", null, headers, 20L));
 
-            inputTopic1.pipeInput("A", null, 15L);
-            inputTopic1.pipeInput("B", null, 20L);
-            proc1.checkAndClearProcessResult(
-                new KeyValueTimestamp<>("A", new Change<>(null, "03"), 15),
-                new KeyValueTimestamp<>("B", new Change<>(null, "02"), 20)
+            if (storeFormat.equals("default")) {
+                proc1.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>("A", new Change<>(null, "03"), 15),
+                    new KeyValueTimestamp<>("B", new Change<>(null, "02"), 20)
+                );
+            } else if (storeFormat.equals("headers")) {
+                proc1.checkAndClearProcessResultWithHeaders(
+                    new KeyValueTimestampHeaders<>("A", new Change<>(null, 
"03"), 15, headers),
+                    new KeyValueTimestampHeaders<>("B", new Change<>(null, 
"02"), 20, headers)
+                );
+            }
+        }
+    }
+
+    @ParameterizedTest
+    @MethodSource("storeFormats")
+    public void testKTableAcceptsInputsWithHeaders(final String storeFormat) {
+        final Properties props = getProps(storeFormat);
+        final StreamsBuilder builder = new StreamsBuilder();
+        final String topic1 = "topic1";
+        final String storeName = "input-headers-store";
+
+        final KTable<String, Integer> table1 = builder.table(topic1,
+            Consumed.with(Serdes.String(), Serdes.Integer()),
+            Materialized.as(storeName));
+
+        final MockApiProcessorSupplier<String, Integer, Void, Void> supplier = 
new MockApiProcessorSupplier<>();
+        table1.toStream().process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            final TestInputTopic<String, Integer> inputTopic =
+                    driver.createInputTopic(topic1, new StringSerializer(), 
new IntegerSerializer());
+
+            final Headers headers1 = makeHeaders("key1", "value1");
+            final Headers headers2 = makeHeaders("key2", "value2");
+
+            inputTopic.pipeInput(new TestRecord<>("A", 1, headers1, 10L));
+            inputTopic.pipeInput(new TestRecord<>("B", 2, headers2, 11L));
+
+            // Headers are forwarded in both modes
+            assertEquals(
+                asList(new KeyValueTimestampHeaders<>("A", 1, 10L, headers1),
+                    new KeyValueTimestampHeaders<>("B", 2, 11L, headers2)),
+                supplier.theCapturedProcessor().processedWithHeaders()
             );
+
+            // Headers are forwarded only in `headers` modes

Review Comment:
   // Headers are forwarded in both modes 
   // Headers are forwarded only in `headers` modes
   This is misleading a bit. I assume in second case this is about saved to 
state store



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueStoreWrapper.java:
##########
@@ -158,4 +176,157 @@ public <R> QueryResult<R> query(final Query<R> query, 
final PositionBound positi
     public Position getPosition() {
         return store.getPosition();
     }
+
+    /**
+     * Adapter that makes a TimestampedKeyValueStore appear as a
+     * TimestampedKeyValueStoreWithHeaders by adding null headers support.
+     */
+    private static final class TimestampedKeyValueStoreHeadersAdapter<K, V>
+        extends WrappedStateStore<TimestampedKeyValueStore<K, V>, K, 
ValueTimestampHeaders<V>>
+        implements TimestampedKeyValueStoreWithHeaders<K, V> {
+
+        public TimestampedKeyValueStoreHeadersAdapter(final 
TimestampedKeyValueStore<K, V> store) {
+            super(store);
+        }
+
+        @Override
+        public void put(final K key, final ValueTimestampHeaders<V> value) {
+            if (value == null) {
+                wrapped().put(key, null);
+            } else {
+                wrapped().put(key, ValueAndTimestamp.make(value.value(), 
value.timestamp()));
+            }
+        }
+
+        @Override
+        public ValueTimestampHeaders<V> putIfAbsent(final K key, final 
ValueTimestampHeaders<V> value) {
+            final ValueAndTimestamp<V> valueAndTimestamp = value == null
+                ? wrapped().putIfAbsent(key, null)
+                : wrapped().putIfAbsent(key, 
ValueAndTimestamp.make(value.value(), value.timestamp()));
+            return valueAndTimestamp == null
+                ? null
+                : ValueTimestampHeaders.make(valueAndTimestamp.value(), 
valueAndTimestamp.timestamp(), null);
+        }
+
+        @Override
+        public void putAll(final List<KeyValue<K, ValueTimestampHeaders<V>>> 
entries) {
+            final List<KeyValue<K, ValueAndTimestamp<V>>> convertedEntries = 
new java.util.ArrayList<>(entries.size());

Review Comment:
   ```suggestion
               final List<KeyValue<K, ValueAndTimestamp<V>>> convertedEntries = 
new ArrayList<>(entries.size());
   ```



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java:
##########
@@ -166,7 +166,12 @@ public void process(final Record<KIn, VIn> record) {
                 } else {
                     oldValue = null;
                 }
-                final long putReturnCode = store.put(record.key(), 
record.value(), record.timestamp());
+                final long putReturnCode;
+//                if (store.supportsHeaders()) {
+                putReturnCode = store.put(record.key(), record.value(), 
record.timestamp(), record.headers());
+//                } else {
+//                    putReturnCode = store.put(record.key(), record.value(), 
record.timestamp());
+//                }

Review Comment:
   Do we want to keep commented code (for follow-up PRs), or it's leftover? 



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