frankvicky commented on code in PR #21386:
URL: https://github.com/apache/kafka/pull/21386#discussion_r2764820963


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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);
+    }
+
+    @Override
+    public ValueTimestampHeaders<V> deserialize(final String topic, final 
byte[] valueTimestampHeaders) {
+        if (valueTimestampHeaders == null) {
+            return null;
+        }
+
+        final ByteBuffer buffer = ByteBuffer.wrap(valueTimestampHeaders);
+
+        // Read header size as varint
+        final int headerSize = ByteUtils.readVarint(buffer);
+
+        // Read headers
+        final byte[] rawHeaders = new byte[headerSize];
+        buffer.get(rawHeaders);
+        final Headers headers = headersDeserializer.deserialize(rawHeaders);
+
+        // Read timestamp (8 bytes)
+        final byte[] rawTimestamp = new byte[Long.BYTES];
+        buffer.get(rawTimestamp);
+        final long timestamp = timestampDeserializer.deserialize(topic, 
rawTimestamp);
+
+        // Read value (remaining bytes)
+        final byte[] rawValue = new byte[buffer.remaining()];
+        buffer.get(rawValue);
+        final V value = valueDeserializer.deserialize(topic, headers, 
rawValue);
+
+        return ValueTimestampHeaders.make(value, timestamp, headers);

Review Comment:
   I think we will need another method like 
`ValueTimestampHeaders#makeWithRawHeaders(V, long, byte[])`.



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