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


##########
streams/src/main/java/org/apache/kafka/streams/state/ValueTimestampHeaders.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import java.util.Objects;
+
+/**
+ * Combines a value with its timestamp and associated record headers.
+ *
+ * @param <V> the value type
+ */
+public final class ValueTimestampHeaders<V> {
+
+    private final V value;
+    private final long timestamp;
+    private Headers headers;
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
Headers headers) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = headers == null ? new RecordHeaders() : headers;
+    }
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
byte[] rawHeaders) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = null;
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null}.
+     *
+     * @param value     the value
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null};
+     * otherwise {@code null} is returned
+     */
+    public static <V> ValueTimestampHeaders<V> make(final V value,
+                                                    final long timestamp,
+                                                    final Headers headers) {
+        if (value == null) {
+            return null;
+        }
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance.
+     * The provided {@code value} may be {@code null}.
+     *
+     * @param value     the value (may be {@code null})
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance
+     */
+    public static <V> ValueTimestampHeaders<V> makeAllowNullable(final V value,
+                                                                 final long 
timestamp,
+                                                                 final Headers 
headers) {
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Return the wrapped {@code value} of the given {@code 
valueTimestampHeaders} parameter
+     * if the parameter is not {@code null}.
+     *
+     * @param valueTimestampHeaders a {@link ValueTimestampHeaders} instance; 
can be {@code null}
+     * @param <V>                   the type of the value
+     * @return the wrapped {@code value} of {@code valueTimestampHeaders} if 
not {@code null}; otherwise {@code null}
+     */
+    public static <V> V getValueOrNull(final ValueTimestampHeaders<V> 
valueTimestampHeaders) {
+        return valueTimestampHeaders == null ? null : 
valueTimestampHeaders.value;
+    }
+
+    /**
+     * <strong>Internal use only.</strong> This method is used by the 
deserialization infrastructure
+     * and should not be called directly by application code.
+     * <p>
+     * Create a new {@link ValueTimestampHeaders} instance with raw 
(serialized) headers for lazy deserialization.
+     * The headers will be deserialized lazily when {@link #headers()} is 
first called, minimizing overhead
+     * during range scans when headers are not accessed.
+     * <p>
+     * This method is used internally by {@link 
org.apache.kafka.streams.state.internals.ValueTimestampHeadersDeserializer}.
+     *
+     * @param value       the value
+     * @param timestamp   the timestamp
+     * @param rawHeaders  the serialized headers bytes
+     * @param <V>         the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null};
+     * otherwise {@code null} is returned
+     */
+    public static <V> ValueTimestampHeaders<V> makeWithRawHeaders(final V 
value,
+                                                                  final long 
timestamp,
+                                                                  final byte[] 
rawHeaders) {
+        if (value == null) {
+            return null;
+        }
+        return new ValueTimestampHeaders<>(value, timestamp, rawHeaders);
+    }
+
+    public V value() {
+        return value;
+    }
+
+    public long timestamp() {
+        return timestamp;
+    }
+
+    public Headers headers() {
+        return headers;
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof ValueTimestampHeaders)) {
+            return false;
+        }
+        final ValueTimestampHeaders<?> that = (ValueTimestampHeaders<?>) o;
+        // Ensure headers are deserialized before comparison (lazy 
deserialization)
+        return timestamp == that.timestamp
+            && Objects.equals(value, that.value)
+            && Objects.equals(this.headers(), that.headers());
+    }
+
+    @Override
+    public int hashCode() {
+        // Ensure headers are deserialized before hashing (lazy 
deserialization)
+        return Objects.hash(value, timestamp, headers());
+    }
+
+    @Override
+    public String toString() {
+        return "ValueTimestampHeaders{" +
+            "value=" + value +
+            ", timestamp=" + timestamp +
+            ", headers=" + headers() +

Review Comment:
   Same.



##########
streams/src/main/java/org/apache/kafka/streams/state/ValueTimestampHeaders.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import java.util.Objects;
+
+/**
+ * Combines a value with its timestamp and associated record headers.
+ *
+ * @param <V> the value type
+ */
+public final class ValueTimestampHeaders<V> {
+
+    private final V value;
+    private final long timestamp;
+    private Headers headers;
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
Headers headers) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = headers == null ? new RecordHeaders() : headers;
+    }
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
byte[] rawHeaders) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = null;
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null}.
+     *
+     * @param value     the value
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null};
+     * otherwise {@code null} is returned
+     */
+    public static <V> ValueTimestampHeaders<V> make(final V value,
+                                                    final long timestamp,
+                                                    final Headers headers) {
+        if (value == null) {
+            return null;
+        }
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance.
+     * The provided {@code value} may be {@code null}.
+     *
+     * @param value     the value (may be {@code null})
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance
+     */
+    public static <V> ValueTimestampHeaders<V> makeAllowNullable(final V value,
+                                                                 final long 
timestamp,
+                                                                 final Headers 
headers) {
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Return the wrapped {@code value} of the given {@code 
valueTimestampHeaders} parameter
+     * if the parameter is not {@code null}.
+     *
+     * @param valueTimestampHeaders a {@link ValueTimestampHeaders} instance; 
can be {@code null}
+     * @param <V>                   the type of the value
+     * @return the wrapped {@code value} of {@code valueTimestampHeaders} if 
not {@code null}; otherwise {@code null}
+     */
+    public static <V> V getValueOrNull(final ValueTimestampHeaders<V> 
valueTimestampHeaders) {
+        return valueTimestampHeaders == null ? null : 
valueTimestampHeaders.value;
+    }
+
+    /**
+     * <strong>Internal use only.</strong> This method is used by the 
deserialization infrastructure
+     * and should not be called directly by application code.
+     * <p>
+     * Create a new {@link ValueTimestampHeaders} instance with raw 
(serialized) headers for lazy deserialization.
+     * The headers will be deserialized lazily when {@link #headers()} is 
first called, minimizing overhead
+     * during range scans when headers are not accessed.
+     * <p>
+     * This method is used internally by {@link 
org.apache.kafka.streams.state.internals.ValueTimestampHeadersDeserializer}.
+     *
+     * @param value       the value
+     * @param timestamp   the timestamp
+     * @param rawHeaders  the serialized headers bytes
+     * @param <V>         the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null};
+     * otherwise {@code null} is returned
+     */
+    public static <V> ValueTimestampHeaders<V> makeWithRawHeaders(final V 
value,
+                                                                  final long 
timestamp,
+                                                                  final byte[] 
rawHeaders) {
+        if (value == null) {
+            return null;
+        }
+        return new ValueTimestampHeaders<>(value, timestamp, rawHeaders);
+    }
+
+    public V value() {
+        return value;
+    }
+
+    public long timestamp() {
+        return timestamp;
+    }
+
+    public Headers headers() {
+        return headers;
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof ValueTimestampHeaders)) {
+            return false;
+        }
+        final ValueTimestampHeaders<?> that = (ValueTimestampHeaders<?>) o;
+        // Ensure headers are deserialized before comparison (lazy 
deserialization)
+        return timestamp == that.timestamp
+            && Objects.equals(value, that.value)
+            && Objects.equals(this.headers(), that.headers());
+    }
+
+    @Override
+    public int hashCode() {
+        // Ensure headers are deserialized before hashing (lazy 
deserialization)
+        return Objects.hash(value, timestamp, headers());

Review Comment:
   Same



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializerTest.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class ValueTimestampHeadersSerializerTest {
+
+    private static final String TOPIC = "test-topic";
+    private static final long TIMESTAMP = 123456789L;
+    private static final String VALUE = "test-value";
+
+    private ValueTimestampHeadersSerializer<String> serializer;
+    private ValueTimestampHeadersDeserializer<String> deserializer;
+
+    @BeforeEach
+    void setup() {
+        serializer = new 
ValueTimestampHeadersSerializer<>(Serdes.String().serializer());
+        deserializer = new 
ValueTimestampHeadersDeserializer<>(Serdes.String().deserializer());
+    }
+
+    @AfterEach
+    void cleanup() {
+        if (serializer != null) {
+            serializer.close();
+        }
+        if (deserializer != null) {
+            deserializer.close();
+        }
+    }
+
+    @Test
+    public void shouldSerializeAndDeserializeNonNullData() {
+        final Headers headers = new RecordHeaders()
+            .add("key1", "value1".getBytes());
+        final ValueTimestampHeaders<String> original =
+            ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        final byte[] serialized = serializer.serialize(TOPIC, original);
+        assertNotNull(serialized);
+
+        final ValueTimestampHeaders<String> deserialized =
+            deserializer.deserialize(TOPIC, serialized);
+
+        assertNotNull(deserialized);
+        assertEquals(original.value(), deserialized.value());
+        assertEquals(original.timestamp(), deserialized.timestamp());
+        assertArrayEquals(original.headers().toArray(), 
deserialized.headers().toArray());
+    }
+
+    @Test
+    public void shouldSerializeNullDataAsNull() {
+        final byte[] serialized = serializer.serialize(TOPIC, null);
+        assertNull(serialized);
+    }
+
+    @Test
+    public void shouldSerializeValueWithEmptyHeaders() {
+        final Headers emptyHeaders = new RecordHeaders();
+        final ValueTimestampHeaders<String> valueTimestampHeaders =
+            ValueTimestampHeaders.make(VALUE, TIMESTAMP, emptyHeaders);
+
+        final byte[] serialized = serializer.serialize(TOPIC, 
valueTimestampHeaders);
+        assertNotNull(serialized);
+
+        final ValueTimestampHeaders<String> deserialized =
+            deserializer.deserialize(TOPIC, serialized);
+
+        assertEquals(VALUE, deserialized.value());
+        assertEquals(TIMESTAMP, deserialized.timestamp());
+        assertEquals(0, deserialized.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldSerializeValueWithMultipleHeaders() {
+        final Headers headers = new RecordHeaders()
+            .add("key1", "value1".getBytes())
+            .add("key1", "value2".getBytes())
+            .add("key3", "value3".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders =
+            ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        final byte[] serialized = serializer.serialize(TOPIC, 
valueTimestampHeaders);
+        assertNotNull(serialized);
+
+        final ValueTimestampHeaders<String> deserialized =
+            deserializer.deserialize(TOPIC, serialized);
+
+        assertEquals(VALUE, deserialized.value());
+        assertEquals(TIMESTAMP, deserialized.timestamp());
+        assertEquals(3, deserialized.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldSerializeValueWithNullHeaders() {

Review Comment:
   Given that `make() replaces `null` with `RecordHeader` (and we should verify 
this in `ValueTimestampHeaderTest`), this test seem not to give any signal, as 
`null` would never reach the serializer?



##########
streams/src/test/java/org/apache/kafka/streams/state/ValueTimestampHeadersTest.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ValueTimestampHeadersTest {
+
+    private static final String VALUE = "test-value";
+    private static final long TIMESTAMP = 123456789L;
+
+    @Test
+    public void shouldCreateInstanceWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldReturnNullWhenValueIsNullWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(null, TIMESTAMP, headers);
+
+        assertNull(valueTimestampHeaders);
+    }
+
+    @Test
+    public void shouldCreateInstanceWithMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateInstanceWithNullValueUsingMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(null, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNull(valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateEmptyHeadersWhenHeadersAreNull() {
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, null);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNotNull(valueTimestampHeaders.headers());
+        assertEquals(0, valueTimestampHeaders.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldGetValueOrNull() {
+        final Headers headers = new RecordHeaders();
+        ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
+        assertNull(ValueTimestampHeaders.getValueOrNull(null));
+
+        valueTimestampHeaders = ValueTimestampHeaders.makeAllowNullable(VALUE, 
TIMESTAMP, null);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
+        assertNull(ValueTimestampHeaders.getValueOrNull(null));

Review Comment:
   Seems to be the first test case so separation it with an empty line. We test 
`getValueOrNull` with object create with `make`, `makeAllowNullable`, and 
`null`-object.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializerTest.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Iterator;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ValueTimestampHeadersDeserializerTest {
+
+    private static final String TOPIC = "test-topic";
+    private ValueTimestampHeadersSerializer<String> serializer;
+    private ValueTimestampHeadersDeserializer<String> deserializer;
+
+    @BeforeEach
+    void setup() {
+        serializer = new 
ValueTimestampHeadersSerializer<>(Serdes.String().serializer());
+        deserializer = new 
ValueTimestampHeadersDeserializer<>(Serdes.String().deserializer());
+    }
+
+    @AfterEach
+    void cleanup() {
+        if (serializer != null) {
+            serializer.close();
+        }
+        if (deserializer != null) {
+            deserializer.close();
+        }
+    }
+
+    @Test
+    public void shouldDeserializeNull() {

Review Comment:
   ```suggestion
       public void shouldDeserializeNullToNull() {
   ```



##########
streams/src/main/java/org/apache/kafka/streams/state/ValueTimestampHeaders.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import java.util.Objects;
+
+/**
+ * Combines a value with its timestamp and associated record headers.
+ *
+ * @param <V> the value type
+ */
+public final class ValueTimestampHeaders<V> {
+
+    private final V value;
+    private final long timestamp;
+    private Headers headers;
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
Headers headers) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = headers == null ? new RecordHeaders() : headers;
+    }
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
byte[] rawHeaders) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = null;
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null}.
+     *
+     * @param value     the value
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null};
+     * otherwise {@code null} is returned
+     */
+    public static <V> ValueTimestampHeaders<V> make(final V value,
+                                                    final long timestamp,
+                                                    final Headers headers) {
+        if (value == null) {
+            return null;
+        }
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance.
+     * The provided {@code value} may be {@code null}.
+     *
+     * @param value     the value (may be {@code null})
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance
+     */
+    public static <V> ValueTimestampHeaders<V> makeAllowNullable(final V value,
+                                                                 final long 
timestamp,
+                                                                 final Headers 
headers) {
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Return the wrapped {@code value} of the given {@code 
valueTimestampHeaders} parameter
+     * if the parameter is not {@code null}.
+     *
+     * @param valueTimestampHeaders a {@link ValueTimestampHeaders} instance; 
can be {@code null}
+     * @param <V>                   the type of the value
+     * @return the wrapped {@code value} of {@code valueTimestampHeaders} if 
not {@code null}; otherwise {@code null}
+     */
+    public static <V> V getValueOrNull(final ValueTimestampHeaders<V> 
valueTimestampHeaders) {
+        return valueTimestampHeaders == null ? null : 
valueTimestampHeaders.value;
+    }
+
+    /**
+     * <strong>Internal use only.</strong> This method is used by the 
deserialization infrastructure
+     * and should not be called directly by application code.
+     * <p>
+     * Create a new {@link ValueTimestampHeaders} instance with raw 
(serialized) headers for lazy deserialization.
+     * The headers will be deserialized lazily when {@link #headers()} is 
first called, minimizing overhead
+     * during range scans when headers are not accessed.
+     * <p>
+     * This method is used internally by {@link 
org.apache.kafka.streams.state.internals.ValueTimestampHeadersDeserializer}.
+     *
+     * @param value       the value
+     * @param timestamp   the timestamp
+     * @param rawHeaders  the serialized headers bytes
+     * @param <V>         the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null};
+     * otherwise {@code null} is returned
+     */
+    public static <V> ValueTimestampHeaders<V> makeWithRawHeaders(final V 
value,
+                                                                  final long 
timestamp,
+                                                                  final byte[] 
rawHeaders) {
+        if (value == null) {
+            return null;
+        }
+        return new ValueTimestampHeaders<>(value, timestamp, rawHeaders);
+    }
+
+    public V value() {
+        return value;
+    }
+
+    public long timestamp() {
+        return timestamp;
+    }
+
+    public Headers headers() {
+        return headers;
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof ValueTimestampHeaders)) {
+            return false;
+        }
+        final ValueTimestampHeaders<?> that = (ValueTimestampHeaders<?>) o;
+        // Ensure headers are deserialized before comparison (lazy 
deserialization)
+        return timestamp == that.timestamp
+            && Objects.equals(value, that.value)
+            && Objects.equals(this.headers(), that.headers());

Review Comment:
   If we remove the "lazy deserialization" from this class, we can use 
`this.headers` members variable directly -- no need to call the getter. (Also 
can remove the comment about it above.)



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.LongSerializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableSerializer;
+
+/**
+ * Serializer for ValueTimestampHeaders.
+ *
+ * Serialization format (per KIP-1271):
+ * [headersSize(varint)][headersBytes][timestamp(8)][value]
+ *
+ * Where:
+ * - headersSize: Size of the headersBytes section in bytes, encoded as varint
+ * - headersBytes:
+ *   - For null/empty headers: headersSize = 0, headersBytes is omitted (0 
bytes)
+ *   - For non-empty headers: headersSize > 0, serialized headers 
([count(varint)][header1][header2]...) from HeadersSerializer
+ * - timestamp: 8-byte long timestamp
+ * - value: Serialized value using the provided value serializer
+ *
+ * This is used by KIP-1271 to serialize values with timestamps and headers 
for state stores.
+ */
+public class ValueTimestampHeadersSerializer<V> implements 
WrappingNullableSerializer<ValueTimestampHeaders<V>, Void, V> {
+    public final Serializer<V> valueSerializer;
+    private final Serializer<Long> timestampSerializer;
+    private final HeadersSerializer headersSerializer;
+
+    ValueTimestampHeadersSerializer(final Serializer<V> valueSerializer) {
+        Objects.requireNonNull(valueSerializer);
+        this.valueSerializer = valueSerializer;
+        this.timestampSerializer = new LongSerializer();
+        this.headersSerializer = new HeadersSerializer();
+    }
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        valueSerializer.configure(configs, isKey);
+        timestampSerializer.configure(configs, isKey);
+        headersSerializer.configure(configs, isKey);
+    }
+
+    @Override
+    public byte[] serialize(final String topic, final ValueTimestampHeaders<V> 
valueTimestampHeaders) {
+        if (valueTimestampHeaders == null) {
+            return null;
+        }
+        return serialize(topic, valueTimestampHeaders.value(), 
valueTimestampHeaders.timestamp(), valueTimestampHeaders.headers());
+    }
+
+    public byte[] serialize(final String topic, final V plainValue, final long 
timestamp, final Headers headers) {

Review Comment:
   nit: should this be `private`?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.LongSerializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableSerializer;
+
+/**
+ * Serializer for ValueTimestampHeaders.
+ *
+ * Serialization format (per KIP-1271):
+ * [headersSize(varint)][headersBytes][timestamp(8)][value]
+ *
+ * Where:
+ * - headersSize: Size of the headersBytes section in bytes, encoded as varint
+ * - headersBytes:
+ *   - For null/empty headers: headersSize = 0, headersBytes is omitted (0 
bytes)
+ *   - For non-empty headers: headersSize > 0, serialized headers 
([count(varint)][header1][header2]...) from HeadersSerializer
+ * - timestamp: 8-byte long timestamp
+ * - value: Serialized value using the provided value serializer
+ *
+ * This is used by KIP-1271 to serialize values with timestamps and headers 
for state stores.
+ */
+public class ValueTimestampHeadersSerializer<V> implements 
WrappingNullableSerializer<ValueTimestampHeaders<V>, Void, V> {
+    public final Serializer<V> valueSerializer;
+    private final Serializer<Long> timestampSerializer;
+    private final HeadersSerializer headersSerializer;
+
+    ValueTimestampHeadersSerializer(final Serializer<V> valueSerializer) {
+        Objects.requireNonNull(valueSerializer);
+        this.valueSerializer = valueSerializer;
+        this.timestampSerializer = new LongSerializer();
+        this.headersSerializer = new HeadersSerializer();
+    }
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        valueSerializer.configure(configs, isKey);
+        timestampSerializer.configure(configs, isKey);
+        headersSerializer.configure(configs, isKey);
+    }
+
+    @Override
+    public byte[] serialize(final String topic, final ValueTimestampHeaders<V> 
valueTimestampHeaders) {
+        if (valueTimestampHeaders == null) {
+            return null;
+        }
+        return serialize(topic, valueTimestampHeaders.value(), 
valueTimestampHeaders.timestamp(), valueTimestampHeaders.headers());
+    }
+
+    public byte[] serialize(final String topic, final V plainValue, final long 
timestamp, final Headers headers) {
+        if (plainValue == null) {
+            return null;
+        }
+
+        final byte[] rawValue = valueSerializer.serialize(topic, headers, 
plainValue);
+
+        // Since we can't control the result of the internal serializer, we 
make sure that the result
+        // is not null as well.
+        // Serializing non-null values to null can be useful when working with 
Optional-like values
+        // where the Optional.empty case is serialized to null.
+        // See the discussion here: https://github.com/apache/kafka/pull/7679
+        if (rawValue == null) {
+            return null;
+        }
+
+        final byte[] rawTimestamp = timestampSerializer.serialize(topic, 
timestamp);
+
+        // empty (byte[0]) for null/empty headers, or 
[count][header1][header2]... for non-empty
+        final byte[] rawHeaders = headersSerializer.serialize(topic, headers);
+
+        // Format: [headersSize(varint)][headersBytes][timestamp(8)][value]
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review Comment:
   nit: add `final` (2x)



##########
streams/src/main/java/org/apache/kafka/streams/state/ValueTimestampHeaders.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import java.util.Objects;
+
+/**
+ * Combines a value with its timestamp and associated record headers.
+ *
+ * @param <V> the value type
+ */
+public final class ValueTimestampHeaders<V> {
+
+    private final V value;
+    private final long timestamp;
+    private Headers headers;
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
Headers headers) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = headers == null ? new RecordHeaders() : headers;
+    }
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
byte[] rawHeaders) {

Review Comment:
   Seem we can remove this constructor?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.LongSerializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableSerializer;
+
+/**
+ * Serializer for ValueTimestampHeaders.
+ *
+ * Serialization format (per KIP-1271):
+ * [headersSize(varint)][headersBytes][timestamp(8)][value]
+ *
+ * Where:
+ * - headersSize: Size of the headersBytes section in bytes, encoded as varint
+ * - headersBytes:
+ *   - For null/empty headers: headersSize = 0, headersBytes is omitted (0 
bytes)
+ *   - For non-empty headers: headersSize > 0, serialized headers 
([count(varint)][header1][header2]...) from HeadersSerializer
+ * - timestamp: 8-byte long timestamp
+ * - value: Serialized value using the provided value serializer
+ *
+ * This is used by KIP-1271 to serialize values with timestamps and headers 
for state stores.
+ */
+public class ValueTimestampHeadersSerializer<V> implements 
WrappingNullableSerializer<ValueTimestampHeaders<V>, Void, V> {
+    public final Serializer<V> valueSerializer;
+    private final Serializer<Long> timestampSerializer;

Review Comment:
   Nit: why `Serializer<Long>` instead of `LongSerializer`?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.LongDeserializer;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer;
+
+/**
+ * Deserializer for ValueTimestampHeaders.
+ *
+ * Deserialization format (per KIP-1271):
+ * [headersSize(varint)][headersBytes][timestamp(8)][value]
+ *
+ * Where:
+ * - headersSize: Size of the headersBytes section in bytes, encoded as varint
+ * - headersBytes:
+ *   - For null/empty headers: headersSize = 0, headersBytes is omitted (0 
bytes)
+ *   - For non-empty headers: headersSize > 0, serialized headers in the 
format [count(varint)][header1][header2]... to be processed by 
HeadersDeserializer.
+ * - timestamp: 8-byte long timestamp
+ * - value: Serialized value to be deserialized with the provided value 
deserializer
+ *
+ * This is used by KIP-1271 to deserialize values with timestamps and headers 
from state stores.
+ */
+class ValueTimestampHeadersDeserializer<V> implements 
WrappingNullableDeserializer<ValueTimestampHeaders<V>, Void, V> {
+    private static final LongDeserializer LONG_DESERIALIZER = new 
LongDeserializer();
+    private static final HeadersDeserializer HEADERS_DESERIALIZER = new 
HeadersDeserializer();
+
+    public final Deserializer<V> valueDeserializer;
+    private final Deserializer<Long> timestampDeserializer;

Review Comment:
   nit: why not `LongDeserializer` ?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.LongDeserializer;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer;
+
+/**
+ * Deserializer for ValueTimestampHeaders.
+ *
+ * Deserialization format (per KIP-1271):
+ * [headersSize(varint)][headersBytes][timestamp(8)][value]
+ *
+ * Where:
+ * - headersSize: Size of the headersBytes section in bytes, encoded as varint
+ * - headersBytes:
+ *   - For null/empty headers: headersSize = 0, headersBytes is omitted (0 
bytes)
+ *   - For non-empty headers: headersSize > 0, serialized headers in the 
format [count(varint)][header1][header2]... to be processed by 
HeadersDeserializer.
+ * - timestamp: 8-byte long timestamp
+ * - value: Serialized value to be deserialized with the provided value 
deserializer
+ *
+ * This is used by KIP-1271 to deserialize values with timestamps and headers 
from state stores.
+ */
+class ValueTimestampHeadersDeserializer<V> implements 
WrappingNullableDeserializer<ValueTimestampHeaders<V>, Void, V> {
+    private static final LongDeserializer LONG_DESERIALIZER = new 
LongDeserializer();
+    private static final HeadersDeserializer HEADERS_DESERIALIZER = new 
HeadersDeserializer();
+
+    public final Deserializer<V> valueDeserializer;
+    private final Deserializer<Long> timestampDeserializer;
+    private final HeadersDeserializer headersDeserializer;
+
+    ValueTimestampHeadersDeserializer(final Deserializer<V> valueDeserializer) 
{
+        Objects.requireNonNull(valueDeserializer);
+        this.valueDeserializer = valueDeserializer;
+        this.timestampDeserializer = new LongDeserializer();
+        this.headersDeserializer = new HeadersDeserializer();
+    }
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        valueDeserializer.configure(configs, isKey);
+        timestampDeserializer.configure(configs, isKey);
+        headersDeserializer.configure(configs, isKey);
+    }
+
+    @Override
+    public ValueTimestampHeaders<V> deserialize(final String topic, final 
byte[] valueTimestampHeaders) {
+        if (valueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(valueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = readBytes(buffer, headersSize);
+        final Headers headers = headersDeserializer.deserialize("", 
rawHeaders);
+        final byte[] rawTimestamp = readBytes(buffer, Long.BYTES);
+        final long timestamp = timestampDeserializer.deserialize(topic, 
rawTimestamp);
+        final byte[] rawValue = readBytes(buffer, buffer.remaining());
+        final V value = valueDeserializer.deserialize(topic, rawValue);
+
+        return ValueTimestampHeaders.make(value, timestamp, headers);
+    }
+
+    @Override
+    public void close() {
+        valueDeserializer.close();
+        timestampDeserializer.close();
+        headersDeserializer.close();
+    }
+
+    @Override
+    public void setIfUnset(final SerdeGetter getter) {
+        // ValueTimestampHeadersDeserializer never wraps a null deserializer 
(or configure would throw),
+        // but it may wrap a deserializer that itself wraps a null 
deserializer.
+        initNullableDeserializer(valueDeserializer, getter);
+    }
+
+    /**
+     * Reads the specified number of bytes from the buffer with validation.
+     *
+     * @param buffer the ByteBuffer to read from
+     * @param length the number of bytes to read
+     * @return the byte array containing the read bytes
+     * @throws SerializationException if buffer doesn't have enough bytes
+     */
+    private static byte[] readBytes(final ByteBuffer buffer, final int length) 
{
+        if (buffer.remaining() < length) {
+            throw new SerializationException(
+                "Invalid ValueTimestampHeaders format: expected " + length +
+                " bytes but only " + buffer.remaining() + " bytes remaining"
+            );
+        }
+        final byte[] bytes = new byte[length];
+        buffer.get(bytes);
+        return bytes;
+    }
+
+    /**
+     * Extract value from serialized ValueTimestampHeaders.
+     */
+    static <V> V value(final byte[] rawValueTimestampHeaders, final 
Deserializer<V> deserializer) {

Review Comment:
   nit: Should we really overload generic type `V` -- We use it already on the 
class, and the type here has nothing to do with the class type, as it's static 
method -- easy to mix up.
   
   Should we use `T` instead to get a better distinction?



##########
streams/src/test/java/org/apache/kafka/streams/state/ValueTimestampHeadersTest.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ValueTimestampHeadersTest {
+
+    private static final String VALUE = "test-value";
+    private static final long TIMESTAMP = 123456789L;
+
+    @Test
+    public void shouldCreateInstanceWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldReturnNullWhenValueIsNullWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(null, TIMESTAMP, headers);
+
+        assertNull(valueTimestampHeaders);
+    }
+
+    @Test
+    public void shouldCreateInstanceWithMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateInstanceWithNullValueUsingMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(null, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNull(valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateEmptyHeadersWhenHeadersAreNull() {
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, null);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNotNull(valueTimestampHeaders.headers());
+        assertEquals(0, valueTimestampHeaders.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldGetValueOrNull() {
+        final Headers headers = new RecordHeaders();
+        ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
+        assertNull(ValueTimestampHeaders.getValueOrNull(null));
+
+        valueTimestampHeaders = ValueTimestampHeaders.makeAllowNullable(VALUE, 
TIMESTAMP, null);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));

Review Comment:
   ```suggestion
           
assertNull(ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
   ```



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializerTest.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class ValueTimestampHeadersSerializerTest {
+
+    private static final String TOPIC = "test-topic";
+    private static final long TIMESTAMP = 123456789L;
+    private static final String VALUE = "test-value";
+
+    private ValueTimestampHeadersSerializer<String> serializer;
+    private ValueTimestampHeadersDeserializer<String> deserializer;
+
+    @BeforeEach
+    void setup() {
+        serializer = new 
ValueTimestampHeadersSerializer<>(Serdes.String().serializer());
+        deserializer = new 
ValueTimestampHeadersDeserializer<>(Serdes.String().deserializer());
+    }
+
+    @AfterEach
+    void cleanup() {
+        if (serializer != null) {
+            serializer.close();
+        }
+        if (deserializer != null) {
+            deserializer.close();
+        }
+    }
+
+    @Test
+    public void shouldSerializeAndDeserializeNonNullData() {
+        final Headers headers = new RecordHeaders()
+            .add("key1", "value1".getBytes());
+        final ValueTimestampHeaders<String> original =
+            ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        final byte[] serialized = serializer.serialize(TOPIC, original);
+        assertNotNull(serialized);
+
+        final ValueTimestampHeaders<String> deserialized =
+            deserializer.deserialize(TOPIC, serialized);
+
+        assertNotNull(deserialized);
+        assertEquals(original.value(), deserialized.value());
+        assertEquals(original.timestamp(), deserialized.timestamp());
+        assertArrayEquals(original.headers().toArray(), 
deserialized.headers().toArray());
+    }
+
+    @Test
+    public void shouldSerializeNullDataAsNull() {
+        final byte[] serialized = serializer.serialize(TOPIC, null);
+        assertNull(serialized);
+    }
+
+    @Test
+    public void shouldSerializeValueWithEmptyHeaders() {
+        final Headers emptyHeaders = new RecordHeaders();
+        final ValueTimestampHeaders<String> valueTimestampHeaders =
+            ValueTimestampHeaders.make(VALUE, TIMESTAMP, emptyHeaders);
+
+        final byte[] serialized = serializer.serialize(TOPIC, 
valueTimestampHeaders);
+        assertNotNull(serialized);
+
+        final ValueTimestampHeaders<String> deserialized =
+            deserializer.deserialize(TOPIC, serialized);
+
+        assertEquals(VALUE, deserialized.value());
+        assertEquals(TIMESTAMP, deserialized.timestamp());
+        assertEquals(0, deserialized.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldSerializeValueWithMultipleHeaders() {
+        final Headers headers = new RecordHeaders()
+            .add("key1", "value1".getBytes())
+            .add("key1", "value2".getBytes())
+            .add("key3", "value3".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders =
+            ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        final byte[] serialized = serializer.serialize(TOPIC, 
valueTimestampHeaders);
+        assertNotNull(serialized);
+
+        final ValueTimestampHeaders<String> deserialized =
+            deserializer.deserialize(TOPIC, serialized);
+
+        assertEquals(VALUE, deserialized.value());
+        assertEquals(TIMESTAMP, deserialized.timestamp());
+        assertEquals(3, deserialized.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldSerializeValueWithNullHeaders() {
+        final ValueTimestampHeaders<String> valueTimestampHeaders =
+            ValueTimestampHeaders.make(VALUE, TIMESTAMP, null);
+
+        final byte[] serialized = serializer.serialize(TOPIC, 
valueTimestampHeaders);
+        assertNotNull(serialized);
+
+        final ValueTimestampHeaders<String> deserialized =
+            deserializer.deserialize(TOPIC, serialized);
+
+        assertEquals(VALUE, deserialized.value());
+        assertEquals(TIMESTAMP, deserialized.timestamp());
+        assertEquals(0, deserialized.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldExtractRawValue() {

Review Comment:
   This seems to test `ValueTimestampHeadersDeserializer.value` -- so it does 
belong here as this is the "serializer test"?
   
   And there is already `shouldExtractValue` in 
`ValueTimestampHeadersDeserializerTest` -- so we can just remove this test?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.LongDeserializer;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer;
+
+/**
+ * Deserializer for ValueTimestampHeaders.
+ *
+ * Deserialization format (per KIP-1271):
+ * [headersSize(varint)][headersBytes][timestamp(8)][value]
+ *
+ * Where:
+ * - headersSize: Size of the headersBytes section in bytes, encoded as varint
+ * - headersBytes:
+ *   - For null/empty headers: headersSize = 0, headersBytes is omitted (0 
bytes)
+ *   - For non-empty headers: headersSize > 0, serialized headers in the 
format [count(varint)][header1][header2]... to be processed by 
HeadersDeserializer.
+ * - timestamp: 8-byte long timestamp
+ * - value: Serialized value to be deserialized with the provided value 
deserializer
+ *
+ * This is used by KIP-1271 to deserialize values with timestamps and headers 
from state stores.
+ */
+class ValueTimestampHeadersDeserializer<V> implements 
WrappingNullableDeserializer<ValueTimestampHeaders<V>, Void, V> {
+    private static final LongDeserializer LONG_DESERIALIZER = new 
LongDeserializer();
+    private static final HeadersDeserializer HEADERS_DESERIALIZER = new 
HeadersDeserializer();
+
+    public final Deserializer<V> valueDeserializer;
+    private final Deserializer<Long> timestampDeserializer;
+    private final HeadersDeserializer headersDeserializer;
+
+    ValueTimestampHeadersDeserializer(final Deserializer<V> valueDeserializer) 
{
+        Objects.requireNonNull(valueDeserializer);
+        this.valueDeserializer = valueDeserializer;
+        this.timestampDeserializer = new LongDeserializer();
+        this.headersDeserializer = new HeadersDeserializer();
+    }
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        valueDeserializer.configure(configs, isKey);
+        timestampDeserializer.configure(configs, isKey);
+        headersDeserializer.configure(configs, isKey);
+    }
+
+    @Override
+    public ValueTimestampHeaders<V> deserialize(final String topic, final 
byte[] valueTimestampHeaders) {
+        if (valueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(valueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = readBytes(buffer, headersSize);
+        final Headers headers = headersDeserializer.deserialize("", 
rawHeaders);

Review Comment:
   ```suggestion
           final Headers headers = headersDeserializer.deserialize(topic, 
rawHeaders);
   ```



##########
streams/src/test/java/org/apache/kafka/streams/state/ValueTimestampHeadersTest.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ValueTimestampHeadersTest {
+
+    private static final String VALUE = "test-value";
+    private static final long TIMESTAMP = 123456789L;
+
+    @Test
+    public void shouldCreateInstanceWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldReturnNullWhenValueIsNullWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(null, TIMESTAMP, headers);
+
+        assertNull(valueTimestampHeaders);
+    }
+
+    @Test
+    public void shouldCreateInstanceWithMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateInstanceWithNullValueUsingMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(null, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNull(valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateEmptyHeadersWhenHeadersAreNull() {
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, null);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNotNull(valueTimestampHeaders.headers());
+        assertEquals(0, valueTimestampHeaders.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldGetValueOrNull() {
+        final Headers headers = new RecordHeaders();
+        ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
+        assertNull(ValueTimestampHeaders.getValueOrNull(null));
+
+        valueTimestampHeaders = ValueTimestampHeaders.makeAllowNullable(VALUE, 
TIMESTAMP, null);

Review Comment:
   ```suggestion
           valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(null, TIMESTAMP, headers);
   ```



##########
streams/src/main/java/org/apache/kafka/streams/state/ValueTimestampHeaders.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.streams.state.internals.HeadersDeserializer;
+
+import java.util.Objects;
+
+/**
+ * Combines a value with its timestamp and associated record headers.
+ *
+ * @param <V> the value type
+ */
+public final class ValueTimestampHeaders<V> {
+
+    private final V value;
+    private final long timestamp;
+    //visible for test
+    volatile Headers headers;
+    private final byte[] rawHeaders;
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
Headers headers) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = headers == null ? new RecordHeaders() : headers;
+        this.rawHeaders = null;
+    }
+
+    private ValueTimestampHeaders(final V value, final long timestamp, final 
byte[] rawHeaders) {
+        this.value = value;
+        this.timestamp = timestamp;
+        this.headers = null;
+        this.rawHeaders = rawHeaders;
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null}.
+     *
+     * @param value     the value
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance if the provided 
{@code value} is not {@code null};
+     * otherwise {@code null} is returned
+     */
+    public static <V> ValueTimestampHeaders<V> make(final V value,
+                                                    final long timestamp,
+                                                    final Headers headers) {
+        if (value == null) {
+            return null;
+        }
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Create a new {@link ValueTimestampHeaders} instance.
+     * The provided {@code value} may be {@code null}.
+     *
+     * @param value     the value (may be {@code null})
+     * @param timestamp the timestamp
+     * @param headers   the headers (may be {@code null}, treated as empty)
+     * @param <V>       the type of the value
+     * @return a new {@link ValueTimestampHeaders} instance
+     */
+    public static <V> ValueTimestampHeaders<V> makeAllowNullable(final V value,
+                                                                 final long 
timestamp,
+                                                                 final Headers 
headers) {
+        return new ValueTimestampHeaders<>(value, timestamp, headers);
+    }
+
+    /**
+     * Return the wrapped {@code value} of the given {@code 
valueTimestampHeaders} parameter
+     * if the parameter is not {@code null}.
+     *
+     * @param valueTimestampHeaders a {@link ValueTimestampHeaders} instance; 
can be {@code null}
+     * @param <V>                   the type of the value
+     * @return the wrapped {@code value} of {@code valueTimestampHeaders} if 
not {@code null}; otherwise {@code null}
+     */
+    public static <V> V getValueOrNull(final ValueTimestampHeaders<V> 
valueTimestampHeaders) {
+        return valueTimestampHeaders == null ? null : 
valueTimestampHeaders.value;
+    }
+
+    /**
+     * <strong>Internal use only.</strong> This method is used by the 
deserialization infrastructure

Review Comment:
   SGTM. Let's remove this method for now to get this PR merged?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.errors.SerializationException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.LongDeserializer;
+import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer;
+
+/**
+ * Deserializer for ValueTimestampHeaders.
+ *
+ * Deserialization format (per KIP-1271):
+ * [headersSize(varint)][headersBytes][timestamp(8)][value]
+ *
+ * Where:
+ * - headersSize: Size of the headersBytes section in bytes, encoded as varint
+ * - headersBytes:
+ *   - For null/empty headers: headersSize = 0, headersBytes is omitted (0 
bytes)
+ *   - For non-empty headers: headersSize > 0, serialized headers in the 
format [count(varint)][header1][header2]... to be processed by 
HeadersDeserializer.
+ * - timestamp: 8-byte long timestamp
+ * - value: Serialized value to be deserialized with the provided value 
deserializer
+ *
+ * This is used by KIP-1271 to deserialize values with timestamps and headers 
from state stores.
+ */
+class ValueTimestampHeadersDeserializer<V> implements 
WrappingNullableDeserializer<ValueTimestampHeaders<V>, Void, V> {
+    private static final LongDeserializer LONG_DESERIALIZER = new 
LongDeserializer();
+    private static final HeadersDeserializer HEADERS_DESERIALIZER = new 
HeadersDeserializer();
+
+    public final Deserializer<V> valueDeserializer;
+    private final Deserializer<Long> timestampDeserializer;
+    private final HeadersDeserializer headersDeserializer;
+
+    ValueTimestampHeadersDeserializer(final Deserializer<V> valueDeserializer) 
{
+        Objects.requireNonNull(valueDeserializer);
+        this.valueDeserializer = valueDeserializer;
+        this.timestampDeserializer = new LongDeserializer();
+        this.headersDeserializer = new HeadersDeserializer();
+    }
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        valueDeserializer.configure(configs, isKey);
+        timestampDeserializer.configure(configs, isKey);
+        headersDeserializer.configure(configs, isKey);
+    }
+
+    @Override
+    public ValueTimestampHeaders<V> deserialize(final String topic, final 
byte[] valueTimestampHeaders) {
+        if (valueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(valueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = readBytes(buffer, headersSize);
+        final Headers headers = headersDeserializer.deserialize("", 
rawHeaders);
+        final byte[] rawTimestamp = readBytes(buffer, Long.BYTES);
+        final long timestamp = timestampDeserializer.deserialize(topic, 
rawTimestamp);
+        final byte[] rawValue = readBytes(buffer, buffer.remaining());
+        final V value = valueDeserializer.deserialize(topic, rawValue);
+
+        return ValueTimestampHeaders.make(value, timestamp, headers);
+    }
+
+    @Override
+    public void close() {
+        valueDeserializer.close();
+        timestampDeserializer.close();
+        headersDeserializer.close();
+    }
+
+    @Override
+    public void setIfUnset(final SerdeGetter getter) {
+        // ValueTimestampHeadersDeserializer never wraps a null deserializer 
(or configure would throw),
+        // but it may wrap a deserializer that itself wraps a null 
deserializer.
+        initNullableDeserializer(valueDeserializer, getter);
+    }
+
+    /**
+     * Reads the specified number of bytes from the buffer with validation.
+     *
+     * @param buffer the ByteBuffer to read from
+     * @param length the number of bytes to read
+     * @return the byte array containing the read bytes
+     * @throws SerializationException if buffer doesn't have enough bytes
+     */
+    private static byte[] readBytes(final ByteBuffer buffer, final int length) 
{
+        if (buffer.remaining() < length) {
+            throw new SerializationException(
+                "Invalid ValueTimestampHeaders format: expected " + length +
+                " bytes but only " + buffer.remaining() + " bytes remaining"
+            );
+        }
+        final byte[] bytes = new byte[length];
+        buffer.get(bytes);
+        return bytes;
+    }
+
+    /**
+     * Extract value from serialized ValueTimestampHeaders.
+     */
+    static <V> V value(final byte[] rawValueTimestampHeaders, final 
Deserializer<V> deserializer) {
+        if (rawValueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+        // skip headers plus timestamp
+        buffer.position(buffer.position() + headersSize + Long.BYTES);
+        byte[] bytes = readBytes(buffer, buffer.remaining());
+
+        return deserializer.deserialize("", bytes);
+    }
+
+    /**
+     * Extract timestamp from serialized ValueTimestampHeaders.
+     */
+    static long timestamp(final byte[] rawValueTimestampHeaders) {
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headersSize = ByteUtils.readVarint(buffer);
+        buffer.position(buffer.position() + headersSize);
+
+        final byte[] rawTimestamp = readBytes(buffer, Long.BYTES);
+        return LONG_DESERIALIZER.deserialize(null, rawTimestamp);

Review Comment:
   ```suggestion
           return LONG_DESERIALIZER.deserialize("", rawTimestamp);
   ```



##########
streams/src/test/java/org/apache/kafka/streams/state/ValueTimestampHeadersTest.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ValueTimestampHeadersTest {
+
+    private static final String VALUE = "test-value";
+    private static final long TIMESTAMP = 123456789L;
+
+    @Test
+    public void shouldCreateInstanceWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldReturnNullWhenValueIsNullWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(null, TIMESTAMP, headers);
+
+        assertNull(valueTimestampHeaders);
+    }
+
+    @Test
+    public void shouldCreateInstanceWithMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateInstanceWithNullValueUsingMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(null, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNull(valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateEmptyHeadersWhenHeadersAreNull() {
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, null);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNotNull(valueTimestampHeaders.headers());
+        assertEquals(0, valueTimestampHeaders.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldGetValueOrNull() {
+        final Headers headers = new RecordHeaders();
+        ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
+        assertNull(ValueTimestampHeaders.getValueOrNull(null));

Review Comment:
   This is the same as line 94 below -- I would remove it here?



##########
streams/src/test/java/org/apache/kafka/streams/state/ValueTimestampHeadersTest.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ValueTimestampHeadersTest {
+
+    private static final String VALUE = "test-value";
+    private static final long TIMESTAMP = 123456789L;
+
+    @Test
+    public void shouldCreateInstanceWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldReturnNullWhenValueIsNullWithMake() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(null, TIMESTAMP, headers);
+
+        assertNull(valueTimestampHeaders);
+    }
+
+    @Test
+    public void shouldCreateInstanceWithMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(VALUE, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertEquals(VALUE, valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateInstanceWithNullValueUsingMakeAllowNullable() {
+        final Headers headers = new RecordHeaders().add("key1", 
"value1".getBytes());
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.makeAllowNullable(null, TIMESTAMP, headers);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNull(valueTimestampHeaders.value());
+        assertEquals(TIMESTAMP, valueTimestampHeaders.timestamp());
+        assertEquals(headers, valueTimestampHeaders.headers());
+    }
+
+    @Test
+    public void shouldCreateEmptyHeadersWhenHeadersAreNull() {
+        final ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, null);
+
+        assertNotNull(valueTimestampHeaders);
+        assertNotNull(valueTimestampHeaders.headers());
+        assertEquals(0, valueTimestampHeaders.headers().toArray().length);
+    }
+
+    @Test
+    public void shouldGetValueOrNull() {
+        final Headers headers = new RecordHeaders();
+        ValueTimestampHeaders<String> valueTimestampHeaders = 
ValueTimestampHeaders.make(VALUE, TIMESTAMP, headers);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
+        assertNull(ValueTimestampHeaders.getValueOrNull(null));
+
+        valueTimestampHeaders = ValueTimestampHeaders.makeAllowNullable(VALUE, 
TIMESTAMP, null);
+        assertEquals(VALUE, 
ValueTimestampHeaders.getValueOrNull(valueTimestampHeaders));
+        assertNull(ValueTimestampHeaders.getValueOrNull(null));

Review Comment:
   ```suggestion
   
           assertNull(ValueTimestampHeaders.getValueOrNull(null));
   ```



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