mjsax commented on code in PR #21497:
URL: https://github.com/apache/kafka/pull/21497#discussion_r2850488592


##########
streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java:
##########
@@ -1100,6 +1109,30 @@ public <K, V> WindowStore<K, ValueAndTimestamp<V>> 
getTimestampedWindowStore(fin
         return store instanceof TimestampedWindowStore ? 
(TimestampedWindowStore<K, V>) store : null;
     }
 
+    /**
+     * Get the {@link TimestampedWindowStoreWithHeaders} with the given name.
+     * The store can be a "regular" or global store.
+     * <p>
+     * This is often useful in test cases to pre-populate the store before the 
test case instructs the topology to
+     * {@link TestInputTopic#pipeInput(TestRecord) process an input message}, 
and/or to check the store afterward.
+     *
+     * @param name the name of the store
+     * @return the key value store, or {@code null} if no {@link 
TimestampedWindowStoreWithHeaders} has been registered with the given name
+     * @see #getAllStateStores()
+     * @see #getStateStore(String)
+     * @see #getKeyValueStore(String)
+     * @see #getTimestampedKeyValueStore(String)
+     * @see #getVersionedKeyValueStore(String)

Review Comment:
   `getTimestampedKeyValueStoreWithHeaders` is missing (guess did not exist yet 
when you create this PR, but was added in the mean time by one of Alieh's PRs)



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedToHeadersWindowStoreAdapter.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.common.TopicPartition;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.query.WindowKeyQuery;
+import org.apache.kafka.streams.query.WindowRangeQuery;
+import org.apache.kafka.streams.query.internals.InternalQueryResultUtil;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedBytesStore;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.TimestampedWindowStoreWithHeaders;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.state.WindowStoreIterator;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import static 
org.apache.kafka.streams.state.HeadersBytesStore.convertToHeaderFormat;
+
+/**
+ * Adapter for backward compatibility between {@link 
TimestampedWindowStoreWithHeaders}
+ * and {@link TimestampedWindowStore}.
+ * <p>
+ * If a user provides a supplier for {@code TimestampedWindowStore} (without 
headers) when building
+ * a {@code TimestampedWindowStoreWithHeaders}, this adapter translates 
between the timestamped
+ * {@code byte[]} format and the timestamped-with-headers {@code byte[]} 
format.
+ * <p>
+ * Format conversion:
+ * <ul>
+ *   <li>Write: {@code [headers][timestamp][value]} → {@code 
[timestamp][value]} (strip headers)</li>
+ *   <li>Read: {@code [timestamp][value]} → {@code 
[headers][timestamp][value]} (add empty headers)</li>
+ * </ul>
+ */
+public class TimestampedToHeadersWindowStoreAdapter implements 
WindowStore<Bytes, byte[]> {
+    private final WindowStore<Bytes, byte[]> store;
+
+    public TimestampedToHeadersWindowStoreAdapter(final WindowStore<Bytes, 
byte[]> store) {
+        if (!store.persistent()) {
+            throw new IllegalArgumentException("Provided store must be a 
persistent store, but it is not.");
+        }
+        if (!(store instanceof TimestampedBytesStore)) {
+            throw new IllegalArgumentException("Provided store must be a 
timestamped store, but it is not.");
+        }
+        this.store = store;
+    }
+
+    /**
+     * Extract raw timestamped value (timestamp + value) from serialized 
ValueTimestampHeaders.
+     * This strips the headers portion but keeps timestamp and value intact.
+     *
+     * Format conversion:
+     * Input:  [headersSize(varint)][headers][timestamp(8)][value]
+     * Output: [timestamp(8)][value]
+     */
+    static byte[] rawTimestampedValue(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+        // Skip headers, keep timestamp + value
+        buffer.position(buffer.position() + headersSize);
+
+        final byte[] result = new byte[buffer.remaining()];
+        buffer.get(result);
+        return result;
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] 
valueWithTimestampAndHeaders, final long windowStartTimestamp) {
+        store.put(key, rawTimestampedValue(valueWithTimestampAndHeaders), 
windowStartTimestamp);
+    }
+
+    @Override
+    public byte[] fetch(final Bytes key, final long timestamp) {
+        return convertToHeaderFormat(store.fetch(key, timestamp));
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> fetch(final Bytes key, final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersWindowStoreIteratorAdapter(store.fetch(key, timeFrom, 
timeTo));
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> backwardFetch(final Bytes key, final 
long timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersWindowStoreIteratorAdapter(store.backwardFetch(key, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes 
keyFrom, final Bytes keyTo,
+                                                           final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.fetch(keyFrom, keyTo, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardFetch(final Bytes 
keyFrom, final Bytes keyTo,
+                                                                   final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardFetch(keyFrom, keyTo, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.fetchAll(timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardFetchAll(final 
long timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardFetchAll(timeFrom, 
timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> all() {
+        return new TimestampedToHeadersKeyValueIteratorAdapter(store.all());
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardAll() {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardAll());
+    }
+
+    @Override
+    public String name() {
+        return store.name();
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        store.init(context, root);
+    }
+
+    @Override
+    public void commit(final Map<TopicPartition, Long> changelogOffsets) {
+        store.commit(changelogOffsets);
+    }
+
+    @Override
+    public void close() {
+        store.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return store.isOpen();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <R> QueryResult<R> query(final Query<R> query,
+                                    final PositionBound positionBound,
+                                    final QueryConfig config) {
+
+        throw new UnsupportedOperationException("Queries (IQv2) are not 
supported for timestamped window stores with headers yet.");
+    }
+
+    @Override
+    public Position getPosition() {
+        return store.getPosition();
+    }
+
+    /**
+     * Iterator adapter for WindowStoreIterator that converts timestamp-only 
values
+     * to timestamp-with-headers format by adding empty headers.
+     */
+    private static class TimestampedToHeadersWindowStoreIteratorAdapter 
implements WindowStoreIterator<byte[]> {
+        private final WindowStoreIterator<byte[]> innerIterator;
+
+        TimestampedToHeadersWindowStoreIteratorAdapter(final 
WindowStoreIterator<byte[]> innerIterator) {
+            this.innerIterator = innerIterator;
+        }
+
+        @Override
+        public void close() {
+            innerIterator.close();
+        }
+
+        @Override
+        public Long peekNextKey() {
+            return innerIterator.peekNextKey();
+        }
+
+        @Override
+        public boolean hasNext() {
+            return innerIterator.hasNext();
+        }
+
+        @Override
+        public KeyValue<Long, byte[]> next() {
+            final KeyValue<Long, byte[]> timestampedKeyValue = 
innerIterator.next();
+            if (timestampedKeyValue == null) {
+                return null;
+            }
+            return KeyValue.pair(timestampedKeyValue.key, 
convertToHeaderFormat(timestampedKeyValue.value));
+        }
+    }
+
+    /**
+     * Iterator adapter for KeyValueIterator that converts timestamp-only 
values
+     * to timestamp-with-headers format by adding empty headers.
+     */
+    private static class TimestampedToHeadersKeyValueIteratorAdapter 
implements KeyValueIterator<Windowed<Bytes>, byte[]> {
+        private final KeyValueIterator<Windowed<Bytes>, byte[]> innerIterator;
+
+        TimestampedToHeadersKeyValueIteratorAdapter(final 
KeyValueIterator<Windowed<Bytes>, byte[]> innerIterator) {
+            this.innerIterator = innerIterator;
+        }
+
+        @Override
+        public void close() {
+            innerIterator.close();
+        }
+
+        @Override
+        public Windowed<Bytes> peekNextKey() {
+            return innerIterator.peekNextKey();
+        }
+
+        @Override
+        public boolean hasNext() {
+            return innerIterator.hasNext();
+        }
+
+        @Override
+        public KeyValue<Windowed<Bytes>, byte[]> next() {
+            final KeyValue<Windowed<Bytes>, byte[]> timestampedKeyValue = 
innerIterator.next();
+            if (timestampedKeyValue == null) {

Review Comment:
   This `null` check is missing in `TimestampedToHeadersIteratorAdapter<K>` -- 
not sure why.
   
   Might be a bug we should address?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedToHeadersWindowStoreAdapter.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.common.TopicPartition;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.query.WindowKeyQuery;
+import org.apache.kafka.streams.query.WindowRangeQuery;
+import org.apache.kafka.streams.query.internals.InternalQueryResultUtil;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedBytesStore;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.TimestampedWindowStoreWithHeaders;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.state.WindowStoreIterator;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import static 
org.apache.kafka.streams.state.HeadersBytesStore.convertToHeaderFormat;
+
+/**
+ * Adapter for backward compatibility between {@link 
TimestampedWindowStoreWithHeaders}
+ * and {@link TimestampedWindowStore}.
+ * <p>
+ * If a user provides a supplier for {@code TimestampedWindowStore} (without 
headers) when building
+ * a {@code TimestampedWindowStoreWithHeaders}, this adapter translates 
between the timestamped
+ * {@code byte[]} format and the timestamped-with-headers {@code byte[]} 
format.
+ * <p>
+ * Format conversion:
+ * <ul>
+ *   <li>Write: {@code [headers][timestamp][value]} → {@code 
[timestamp][value]} (strip headers)</li>
+ *   <li>Read: {@code [timestamp][value]} → {@code 
[headers][timestamp][value]} (add empty headers)</li>
+ * </ul>
+ */
+public class TimestampedToHeadersWindowStoreAdapter implements 
WindowStore<Bytes, byte[]> {
+    private final WindowStore<Bytes, byte[]> store;
+
+    public TimestampedToHeadersWindowStoreAdapter(final WindowStore<Bytes, 
byte[]> store) {
+        if (!store.persistent()) {
+            throw new IllegalArgumentException("Provided store must be a 
persistent store, but it is not.");
+        }
+        if (!(store instanceof TimestampedBytesStore)) {
+            throw new IllegalArgumentException("Provided store must be a 
timestamped store, but it is not.");
+        }
+        this.store = store;
+    }
+
+    /**
+     * Extract raw timestamped value (timestamp + value) from serialized 
ValueTimestampHeaders.
+     * This strips the headers portion but keeps timestamp and value intact.
+     *
+     * Format conversion:
+     * Input:  [headersSize(varint)][headers][timestamp(8)][value]
+     * Output: [timestamp(8)][value]
+     */
+    static byte[] rawTimestampedValue(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+        // Skip headers, keep timestamp + value
+        buffer.position(buffer.position() + headersSize);
+
+        final byte[] result = new byte[buffer.remaining()];
+        buffer.get(result);
+        return result;
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] 
valueWithTimestampAndHeaders, final long windowStartTimestamp) {
+        store.put(key, rawTimestampedValue(valueWithTimestampAndHeaders), 
windowStartTimestamp);
+    }
+
+    @Override
+    public byte[] fetch(final Bytes key, final long timestamp) {
+        return convertToHeaderFormat(store.fetch(key, timestamp));
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> fetch(final Bytes key, final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersWindowStoreIteratorAdapter(store.fetch(key, timeFrom, 
timeTo));
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> backwardFetch(final Bytes key, final 
long timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersWindowStoreIteratorAdapter(store.backwardFetch(key, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes 
keyFrom, final Bytes keyTo,
+                                                           final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.fetch(keyFrom, keyTo, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardFetch(final Bytes 
keyFrom, final Bytes keyTo,
+                                                                   final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardFetch(keyFrom, keyTo, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.fetchAll(timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardFetchAll(final 
long timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardFetchAll(timeFrom, 
timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> all() {
+        return new TimestampedToHeadersKeyValueIteratorAdapter(store.all());
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardAll() {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardAll());
+    }
+
+    @Override
+    public String name() {
+        return store.name();
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        store.init(context, root);
+    }
+
+    @Override
+    public void commit(final Map<TopicPartition, Long> changelogOffsets) {
+        store.commit(changelogOffsets);
+    }
+
+    @Override
+    public void close() {
+        store.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return store.isOpen();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <R> QueryResult<R> query(final Query<R> query,
+                                    final PositionBound positionBound,
+                                    final QueryConfig config) {
+
+        throw new UnsupportedOperationException("Queries (IQv2) are not 
supported for timestamped window stores with headers yet.");
+    }
+
+    @Override
+    public Position getPosition() {
+        return store.getPosition();
+    }
+
+    /**
+     * Iterator adapter for WindowStoreIterator that converts timestamp-only 
values
+     * to timestamp-with-headers format by adding empty headers.
+     */
+    private static class TimestampedToHeadersWindowStoreIteratorAdapter 
implements WindowStoreIterator<byte[]> {

Review Comment:
   It seems this one is very similar to 
`KeyValueToTimestampedKeyValueIteratorAdapter` just with the difference of the 
"converter function".
   
   If we make the "converter function" a parameter, we would unify the code and 
just pass in different function.
   
   Let's file a ticket to do this unification to reduce duplicated code. We 
should file this ticket not limited for this case, but look across all of the 
adaptor code, to see what we can unify. This might not be the only case of 
unnecessary code duplication.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedToHeadersWindowStoreAdapter.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.common.TopicPartition;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.query.WindowKeyQuery;
+import org.apache.kafka.streams.query.WindowRangeQuery;
+import org.apache.kafka.streams.query.internals.InternalQueryResultUtil;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedBytesStore;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.TimestampedWindowStoreWithHeaders;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.state.WindowStoreIterator;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import static 
org.apache.kafka.streams.state.HeadersBytesStore.convertToHeaderFormat;
+
+/**
+ * Adapter for backward compatibility between {@link 
TimestampedWindowStoreWithHeaders}
+ * and {@link TimestampedWindowStore}.
+ * <p>
+ * If a user provides a supplier for {@code TimestampedWindowStore} (without 
headers) when building
+ * a {@code TimestampedWindowStoreWithHeaders}, this adapter translates 
between the timestamped
+ * {@code byte[]} format and the timestamped-with-headers {@code byte[]} 
format.
+ * <p>
+ * Format conversion:
+ * <ul>
+ *   <li>Write: {@code [headers][timestamp][value]} → {@code 
[timestamp][value]} (strip headers)</li>
+ *   <li>Read: {@code [timestamp][value]} → {@code 
[headers][timestamp][value]} (add empty headers)</li>
+ * </ul>
+ */
+public class TimestampedToHeadersWindowStoreAdapter implements 
WindowStore<Bytes, byte[]> {
+    private final WindowStore<Bytes, byte[]> store;
+
+    public TimestampedToHeadersWindowStoreAdapter(final WindowStore<Bytes, 
byte[]> store) {
+        if (!store.persistent()) {
+            throw new IllegalArgumentException("Provided store must be a 
persistent store, but it is not.");
+        }
+        if (!(store instanceof TimestampedBytesStore)) {
+            throw new IllegalArgumentException("Provided store must be a 
timestamped store, but it is not.");
+        }
+        this.store = store;
+    }
+
+    /**
+     * Extract raw timestamped value (timestamp + value) from serialized 
ValueTimestampHeaders.
+     * This strips the headers portion but keeps timestamp and value intact.
+     *
+     * Format conversion:
+     * Input:  [headersSize(varint)][headers][timestamp(8)][value]
+     * Output: [timestamp(8)][value]
+     */
+    static byte[] rawTimestampedValue(final byte[] rawValueTimestampHeaders) {

Review Comment:
   This is the exact some code as we have in 
`TimestampedToHeadersStoreAdapter`. Can you unify it?
   
   I know that there is also a ticket about adding some utility class, so we 
could also do this as part of this ticket -- but we might want to add a comment 
there for tracking? Otherwise, we might miss something?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedToHeadersWindowStoreAdapter.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.common.TopicPartition;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.query.WindowKeyQuery;
+import org.apache.kafka.streams.query.WindowRangeQuery;
+import org.apache.kafka.streams.query.internals.InternalQueryResultUtil;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedBytesStore;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.TimestampedWindowStoreWithHeaders;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.state.WindowStoreIterator;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import static 
org.apache.kafka.streams.state.HeadersBytesStore.convertToHeaderFormat;
+
+/**
+ * Adapter for backward compatibility between {@link 
TimestampedWindowStoreWithHeaders}
+ * and {@link TimestampedWindowStore}.
+ * <p>
+ * If a user provides a supplier for {@code TimestampedWindowStore} (without 
headers) when building
+ * a {@code TimestampedWindowStoreWithHeaders}, this adapter translates 
between the timestamped
+ * {@code byte[]} format and the timestamped-with-headers {@code byte[]} 
format.
+ * <p>
+ * Format conversion:
+ * <ul>
+ *   <li>Write: {@code [headers][timestamp][value]} → {@code 
[timestamp][value]} (strip headers)</li>
+ *   <li>Read: {@code [timestamp][value]} → {@code 
[headers][timestamp][value]} (add empty headers)</li>
+ * </ul>
+ */
+public class TimestampedToHeadersWindowStoreAdapter implements 
WindowStore<Bytes, byte[]> {
+    private final WindowStore<Bytes, byte[]> store;
+
+    public TimestampedToHeadersWindowStoreAdapter(final WindowStore<Bytes, 
byte[]> store) {
+        if (!store.persistent()) {
+            throw new IllegalArgumentException("Provided store must be a 
persistent store, but it is not.");
+        }
+        if (!(store instanceof TimestampedBytesStore)) {
+            throw new IllegalArgumentException("Provided store must be a 
timestamped store, but it is not.");
+        }
+        this.store = store;
+    }
+
+    /**
+     * Extract raw timestamped value (timestamp + value) from serialized 
ValueTimestampHeaders.
+     * This strips the headers portion but keeps timestamp and value intact.
+     *
+     * Format conversion:
+     * Input:  [headersSize(varint)][headers][timestamp(8)][value]
+     * Output: [timestamp(8)][value]
+     */
+    static byte[] rawTimestampedValue(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+        // Skip headers, keep timestamp + value
+        buffer.position(buffer.position() + headersSize);
+
+        final byte[] result = new byte[buffer.remaining()];
+        buffer.get(result);
+        return result;
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] 
valueWithTimestampAndHeaders, final long windowStartTimestamp) {
+        store.put(key, rawTimestampedValue(valueWithTimestampAndHeaders), 
windowStartTimestamp);
+    }
+
+    @Override
+    public byte[] fetch(final Bytes key, final long timestamp) {
+        return convertToHeaderFormat(store.fetch(key, timestamp));
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> fetch(final Bytes key, final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersWindowStoreIteratorAdapter(store.fetch(key, timeFrom, 
timeTo));
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> backwardFetch(final Bytes key, final 
long timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersWindowStoreIteratorAdapter(store.backwardFetch(key, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes 
keyFrom, final Bytes keyTo,
+                                                           final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.fetch(keyFrom, keyTo, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardFetch(final Bytes 
keyFrom, final Bytes keyTo,
+                                                                   final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardFetch(keyFrom, keyTo, 
timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final long 
timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.fetchAll(timeFrom, timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardFetchAll(final 
long timeFrom, final long timeTo) {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardFetchAll(timeFrom, 
timeTo));
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> all() {
+        return new TimestampedToHeadersKeyValueIteratorAdapter(store.all());
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> backwardAll() {
+        return new 
TimestampedToHeadersKeyValueIteratorAdapter(store.backwardAll());
+    }
+
+    @Override
+    public String name() {
+        return store.name();
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        store.init(context, root);
+    }
+
+    @Override
+    public void commit(final Map<TopicPartition, Long> changelogOffsets) {
+        store.commit(changelogOffsets);
+    }
+
+    @Override
+    public void close() {
+        store.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return store.isOpen();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <R> QueryResult<R> query(final Query<R> query,
+                                    final PositionBound positionBound,
+                                    final QueryConfig config) {
+
+        throw new UnsupportedOperationException("Queries (IQv2) are not 
supported for timestamped window stores with headers yet.");
+    }
+
+    @Override
+    public Position getPosition() {
+        return store.getPosition();
+    }
+
+    /**
+     * Iterator adapter for WindowStoreIterator that converts timestamp-only 
values
+     * to timestamp-with-headers format by adding empty headers.
+     */
+    private static class TimestampedToHeadersWindowStoreIteratorAdapter 
implements WindowStoreIterator<byte[]> {
+        private final WindowStoreIterator<byte[]> innerIterator;
+
+        TimestampedToHeadersWindowStoreIteratorAdapter(final 
WindowStoreIterator<byte[]> innerIterator) {
+            this.innerIterator = innerIterator;
+        }
+
+        @Override
+        public void close() {
+            innerIterator.close();
+        }
+
+        @Override
+        public Long peekNextKey() {
+            return innerIterator.peekNextKey();
+        }
+
+        @Override
+        public boolean hasNext() {
+            return innerIterator.hasNext();
+        }
+
+        @Override
+        public KeyValue<Long, byte[]> next() {
+            final KeyValue<Long, byte[]> timestampedKeyValue = 
innerIterator.next();
+            if (timestampedKeyValue == null) {
+                return null;
+            }
+            return KeyValue.pair(timestampedKeyValue.key, 
convertToHeaderFormat(timestampedKeyValue.value));
+        }
+    }
+
+    /**
+     * Iterator adapter for KeyValueIterator that converts timestamp-only 
values
+     * to timestamp-with-headers format by adding empty headers.
+     */
+    private static class TimestampedToHeadersKeyValueIteratorAdapter 
implements KeyValueIterator<Windowed<Bytes>, byte[]> {

Review Comment:
   Do we need this class? Can we re-use 
`TimestampedToHeadersIteratorAdapter<K>` ?



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