gensericghiro commented on code in PR #21409: URL: https://github.com/apache/kafka/pull/21409#discussion_r2775342216
########## 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 + * 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) { + synchronized (this) { + if (headers == null) { Review Comment: This is already checked on line 134, why do we need to check it again here? ########## streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java: ########## @@ -0,0 +1,162 @@ +/* + * 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 headersSize = ByteUtils.readVarint(buffer); + + final byte[] rawHeaders = readBytes(buffer, headersSize); + 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.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); + } + + /** + * 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 IllegalArgumentException if buffer doesn't have enough bytes + */ + private static byte[] readBytes(final ByteBuffer buffer, final int length) { + if (buffer.remaining() < length) { + throw new IllegalArgumentException( + "Invalid ValueTimestampHeaders format: expected " + length + + " bytes but only " + buffer.remaining() + " bytes remaining" + ); + } + final byte[] bytes = new byte[length]; + buffer.get(bytes); + return bytes; + } + + /** + * 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 headersSize = ByteUtils.readVarint(buffer); + buffer.position(buffer.position() + headersSize + Long.BYTES); + + return readBytes(buffer, buffer.remaining()); + } + + /** + * Extract timestamp from serialized ValueTimestampHeaders. + */ + static long timestamp(final byte[] rawValueTimestampHeaders) { Review Comment: Don't we need to check for nullity here as well like in the other methods, i.e. `rawValue` and `headers`? ########## streams/src/test/java/org/apache/kafka/streams/state/internals/HeadersSerializerTest.java: ########## @@ -131,29 +129,4 @@ public void shouldSerializeHeadersWithEmptyValue() { assertEquals("key1", header.key()); assertArrayEquals(new byte[0], header.value()); } - - @Test - public void shouldSerializeHeadersWithSpecialCharacters() { Review Comment: I'm fairly new to this, so I might be missing context, but why are we removing this test? Are we not supporting special characters moving forward? -- 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]
