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


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.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.internals;
+
+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: Serialized headers ([count(varint)][header1][header2]...) 
to be deserialized 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();
+
+    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 headerSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);
+
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);
+
+        final long timestamp = timestampDeserializer.deserialize(topic, 
rawTimestamp);
+
+        final byte[] rawValue = new byte[buffer.remaining()];
+        buffer.get(rawValue);
+
+        final V value = valueDeserializer.deserialize(topic, rawValue);
+
+        return ValueTimestampHeaders.makeWithRawHeaders(value, timestamp, 
rawHeaders);
+    }
+
+    @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);
+    }
+
+    /**
+     * Extract raw value bytes from serialized ValueTimestampHeaders.
+     */
+    static byte[] rawValue(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headerSize = ByteUtils.readVarint(buffer);
+        buffer.position(buffer.position() + headerSize + Long.BYTES);
+
+        final byte[] rawValue = new byte[buffer.remaining()];
+        buffer.get(rawValue);
+
+        return rawValue;
+    }
+
+    /**
+     * Extract timestamp from serialized ValueTimestampHeaders.
+     */
+    static long timestamp(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            throw new IllegalArgumentException("Cannot extract timestamp from 
null data");
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headerSize = ByteUtils.readVarint(buffer);
+        buffer.position(buffer.position() + headerSize);
+
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);
+
+        return LONG_DESERIALIZER.deserialize(null, rawTimestamp);
+    }
+
+    /**
+     * Extract headers from serialized ValueTimestampHeaders.
+     */
+    static Headers headers(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            throw new IllegalArgumentException("Cannot extract headers from 
null data");
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headerSize = ByteUtils.readVarint(buffer);

Review Comment:
   var name `headersSize`?



##########
streams/src/main/java/org/apache/kafka/streams/state/ValueTimestampHeaders.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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;
+    private 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
+     * 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() {
+        if (headers == null && rawHeaders != null) {

Review Comment:
   wondering whether we need thread safety here? What if multiple threads see 
that `(headers == null && rawHeaders != null)` and they all do the 
derserialization here?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.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.internals;
+
+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: Serialized headers ([count(varint)][header1][header2]...) 
to be deserialized 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();
+
+    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 headerSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);

Review Comment:
   should we add validation to prevent BufferUnderflowException?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.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.internals;
+
+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: Serialized headers ([count(varint)][header1][header2]...) 
to be deserialized 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();
+
+    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 headerSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);
+
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);

Review Comment:
   the same here



##########
streams/src/main/java/org/apache/kafka/streams/state/ValueTimestampHeaders.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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;
+    private 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
+     * 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() {
+        if (headers == null && rawHeaders != null) {
+            headers = HeadersDeserializer.deserialize(rawHeaders);
+        }
+        return headers != null ? headers : new RecordHeaders();

Review Comment:
   if `headers==null`, then every call to `headers()` creates a new 
`RecordHeaders()` instance. This violates the expected behavior that headers() 
returns the same object.
   ```
   public Headers headers() {
         if (headers == null) {
             if (rawHeaders != null) {
                 headers = HeadersDeserializer.deserialize(rawHeaders);
             } else {
                 headers = new RecordHeaders();
             }
         }
         return headers;
     }
   ```



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.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.internals;
+
+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: Serialized headers ([count(varint)][header1][header2]...) 
to be deserialized 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();
+
+    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 headerSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);
+
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);
+
+        final long timestamp = timestampDeserializer.deserialize(topic, 
rawTimestamp);
+
+        final byte[] rawValue = new byte[buffer.remaining()];
+        buffer.get(rawValue);
+
+        final V value = valueDeserializer.deserialize(topic, rawValue);
+
+        return ValueTimestampHeaders.makeWithRawHeaders(value, timestamp, 
rawHeaders);
+    }
+
+    @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);
+    }
+
+    /**
+     * Extract raw value bytes from serialized ValueTimestampHeaders.
+     */
+    static byte[] rawValue(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headerSize = ByteUtils.readVarint(buffer);
+        buffer.position(buffer.position() + headerSize + Long.BYTES);
+
+        final byte[] rawValue = new byte[buffer.remaining()];
+        buffer.get(rawValue);
+
+        return rawValue;
+    }
+
+    /**
+     * Extract timestamp from serialized ValueTimestampHeaders.
+     */
+    static long timestamp(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            throw new IllegalArgumentException("Cannot extract timestamp from 
null data");
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headerSize = ByteUtils.readVarint(buffer);
+        buffer.position(buffer.position() + headerSize);
+
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);
+
+        return LONG_DESERIALIZER.deserialize(null, rawTimestamp);
+    }
+
+    /**
+     * Extract headers from serialized ValueTimestampHeaders.
+     */
+    static Headers headers(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            throw new IllegalArgumentException("Cannot extract headers from 
null data");
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(rawValueTimestampHeaders);
+        final int headerSize = ByteUtils.readVarint(buffer);
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);

Review Comment:
   should we add validation to prevent `BufferUnderflowException`?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.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.internals;
+
+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: Serialized headers ([count(varint)][header1][header2]...) 
to be deserialized 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();
+
+    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 headerSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);
+
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);
+
+        final long timestamp = timestampDeserializer.deserialize(topic, 
rawTimestamp);
+
+        final byte[] rawValue = new byte[buffer.remaining()];
+        buffer.get(rawValue);

Review Comment:
   the same here
   



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.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.internals;
+
+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: Serialized headers ([count(varint)][header1][header2]...) 
to be deserialized 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();
+
+    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 headerSize = ByteUtils.readVarint(buffer);
+
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);
+
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);
+
+        final long timestamp = timestampDeserializer.deserialize(topic, 
rawTimestamp);
+
+        final byte[] rawValue = new byte[buffer.remaining()];
+        buffer.get(rawValue);
+
+        final V value = valueDeserializer.deserialize(topic, rawValue);
+
+        return ValueTimestampHeaders.makeWithRawHeaders(value, timestamp, 
rawHeaders);
+    }
+
+    @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);
+    }
+
+    /**
+     * Extract raw value bytes from serialized ValueTimestampHeaders.
+     */
+    static byte[] rawValue(final byte[] rawValueTimestampHeaders) {
+        if (rawValueTimestampHeaders == null) {
+            return null;

Review Comment:
   Is that intentional that  `timestamp(null)` and `headers(null)` throw 
IllegalArgumentException but here we return null?



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializerTest.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.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 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;
+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();
+        }
+    }
+

Review Comment:
   Could we verify that headers are actually deserialized lazily?
   



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersSerializer.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.Header;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.ByteUtils;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Serializer for Kafka Headers.
+ * <p>
+ * Serialization format (per KIP-1271):
+ * [NumHeaders(varint)][Header1][Header2]...
+ * <p>
+ * Each header:
+ * [KeyLength(varint)][KeyBytes(UTF-8)][ValueLength(varint)][ValueBytes]
+ * <p>
+ * Note: ValueLength is -1 for null values (encoded as varint).
+ * All integers are encoded as varints (signed varint encoding).
+ * <p>
+ * This serializer produces the headersBytes portion. The headersSize prefix
+ * is added by the outer serializer (e.g., ValueTimestampHeadersSerializer).
+ * <p>
+ * This is used by KIP-1271 to serialize headers for storage in state stores.
+ */
+public class HeadersSerializer implements Serializer<Headers> {
+
+    /**
+     * Serializes headers into a byte array using varint encoding per KIP-1271.
+     * <p>
+     * The output format is [count][header1][header2]... without a size prefix.
+     * The size prefix is added by the outer serializer that uses this.
+     *
+     * @param topic topic associated with data
+     * @param headers the headers to serialize (can be null)
+     * @return the serialized byte array
+     */
+    @Override
+    public byte[] serialize(final String topic, final Headers headers) {
+        try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+             final DataOutputStream out = new DataOutputStream(baos)) {
+
+            final Header[] headerArray = (headers == null) ? new Header[0] : 
headers.toArray();
+            ByteUtils.writeVarint(headerArray.length, out);
+
+            for (final Header header : headerArray) {
+                final byte[] keyBytes = 
header.key().getBytes(StandardCharsets.UTF_8);

Review Comment:
   What if `header.key()` returns `null`? this will throw NPE. Does the Kafka 
Header interface guarantee non-null keys?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.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: 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> 
data) {
+        if (data == null) {
+            return null;
+        }
+        return serialize(topic, data.value(), data.timestamp(), 
data.headers());
+    }
+
+    public byte[] serialize(final String topic, final V data, final long 
timestamp, final Headers headers) {
+        if (data == null) {
+            return null;
+        }
+
+        final byte[] rawValue = valueSerializer.serialize(topic, headers, 
data);
+
+        // 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[] rawHeaders = headersSerializer.serialize(topic, headers); 
 // [count][header1][header2]...
+        final byte[] rawTimestamp = timestampSerializer.serialize(topic, 
timestamp);
+
+        // Format: [headersSize(varint)][headersBytes][timestamp(8)][value]
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+             DataOutputStream out = new DataOutputStream(baos)) {
+
+            ByteUtils.writeVarint(rawHeaders.length, out);  // headersSize
+            out.write(rawHeaders);                           // 
[count][header1][header2]...
+            out.write(rawTimestamp);                         // [timestamp(8)]
+            out.write(rawValue);                             // [value]
+
+            return baos.toByteArray();
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to serialize 
ValueTimestampHeaders", e);

Review Comment:
   Should it be more specific? `SerializationException` instead of 
`RuntimeException`?



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