aliehsaeedii commented on code in PR #21511: URL: https://github.com/apache/kafka/pull/21511#discussion_r2832393068
########## streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersDeserializer.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.utils.ByteUtils; +import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.state.AggregationWithHeaders; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer; + +/** + * Deserializer for AggregationWithHeaders. + * Deserialization format (per KIP-1271): + * [headersSize(varint)][headersBytes][aggregation] + * <p> + * 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. + * - aggregation: Serialized aggregation to be deserialized with the provided aggregation deserializer + * <p> + * This is used by KIP-1271 to deserialize aggregations with headers from session state stores. + */ +class AggregationWithHeadersDeserializer<AGG> implements WrappingNullableDeserializer<AggregationWithHeaders<AGG>, Void, AGG> { + private static final HeadersDeserializer HEADERS_DESERIALIZER = new HeadersDeserializer(); + + public final Deserializer<AGG> aggregationDeserializer; + private final HeadersDeserializer headersDeserializer; + + AggregationWithHeadersDeserializer(final Deserializer<AGG> aggregationDeserializer) { + Objects.requireNonNull(aggregationDeserializer); + this.aggregationDeserializer = aggregationDeserializer; + this.headersDeserializer = new HeadersDeserializer(); + } + + @Override + public void configure(final Map<String, ?> configs, final boolean isKey) { + aggregationDeserializer.configure(configs, isKey); + headersDeserializer.configure(configs, isKey); + } + + @Override + public AggregationWithHeaders<AGG> deserialize(final String topic, final byte[] aggregationWithHeaders) { + if (aggregationWithHeaders == null) { + return null; + } + + final ByteBuffer buffer = ByteBuffer.wrap(aggregationWithHeaders); + final int headersSize = ByteUtils.readVarint(buffer); + + final byte[] rawHeaders = readBytes(buffer, headersSize); + final Headers headers = headersDeserializer.deserialize(topic, rawHeaders); + final byte[] rawAggregation = readBytes(buffer, buffer.remaining()); + final AGG aggregation = aggregationDeserializer.deserialize(topic, headers, rawAggregation); + + return AggregationWithHeaders.make(aggregation, headers); + } + + @Override + public void close() { + aggregationDeserializer.close(); + headersDeserializer.close(); + } + + @Override + public void setIfUnset(final SerdeGetter getter) { + initNullableDeserializer(aggregationDeserializer, 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 or length is negative + */ + private static byte[] readBytes(final ByteBuffer buffer, final int length) { Review Comment: This may not be directly related to this PR, but we could refactor the code so that this method lives in a shared place and can be reused by other classes as well. ########## streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersDeserializer.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.utils.ByteUtils; +import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.state.AggregationWithHeaders; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer; + +/** + * Deserializer for AggregationWithHeaders. + * Deserialization format (per KIP-1271): + * [headersSize(varint)][headersBytes][aggregation] + * <p> + * 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. + * - aggregation: Serialized aggregation to be deserialized with the provided aggregation deserializer + * <p> + * This is used by KIP-1271 to deserialize aggregations with headers from session state stores. + */ +class AggregationWithHeadersDeserializer<AGG> implements WrappingNullableDeserializer<AggregationWithHeaders<AGG>, Void, AGG> { + private static final HeadersDeserializer HEADERS_DESERIALIZER = new HeadersDeserializer(); + + public final Deserializer<AGG> aggregationDeserializer; + private final HeadersDeserializer headersDeserializer; + + AggregationWithHeadersDeserializer(final Deserializer<AGG> aggregationDeserializer) { + Objects.requireNonNull(aggregationDeserializer); + this.aggregationDeserializer = aggregationDeserializer; + this.headersDeserializer = new HeadersDeserializer(); + } + + @Override + public void configure(final Map<String, ?> configs, final boolean isKey) { + aggregationDeserializer.configure(configs, isKey); + headersDeserializer.configure(configs, isKey); + } + + @Override + public AggregationWithHeaders<AGG> deserialize(final String topic, final byte[] aggregationWithHeaders) { + if (aggregationWithHeaders == null) { + return null; + } + + final ByteBuffer buffer = ByteBuffer.wrap(aggregationWithHeaders); + final int headersSize = ByteUtils.readVarint(buffer); + + final byte[] rawHeaders = readBytes(buffer, headersSize); + final Headers headers = headersDeserializer.deserialize(topic, rawHeaders); + final byte[] rawAggregation = readBytes(buffer, buffer.remaining()); + final AGG aggregation = aggregationDeserializer.deserialize(topic, headers, rawAggregation); + + return AggregationWithHeaders.make(aggregation, headers); + } + + @Override + public void close() { + aggregationDeserializer.close(); + headersDeserializer.close(); + } + + @Override + public void setIfUnset(final SerdeGetter getter) { + initNullableDeserializer(aggregationDeserializer, 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 or length is negative + */ + private static byte[] readBytes(final ByteBuffer buffer, final int length) { + if (length < 0) { Review Comment: thanks. I like this check. We dont have it in other places. ########## streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersDeserializer.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.utils.ByteUtils; +import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.state.AggregationWithHeaders; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer; + +/** + * Deserializer for AggregationWithHeaders. + * Deserialization format (per KIP-1271): + * [headersSize(varint)][headersBytes][aggregation] + * <p> + * 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. + * - aggregation: Serialized aggregation to be deserialized with the provided aggregation deserializer + * <p> + * This is used by KIP-1271 to deserialize aggregations with headers from session state stores. + */ +class AggregationWithHeadersDeserializer<AGG> implements WrappingNullableDeserializer<AggregationWithHeaders<AGG>, Void, AGG> { + private static final HeadersDeserializer HEADERS_DESERIALIZER = new HeadersDeserializer(); + + public final Deserializer<AGG> aggregationDeserializer; + private final HeadersDeserializer headersDeserializer; + + AggregationWithHeadersDeserializer(final Deserializer<AGG> aggregationDeserializer) { + Objects.requireNonNull(aggregationDeserializer); + this.aggregationDeserializer = aggregationDeserializer; + this.headersDeserializer = new HeadersDeserializer(); + } + + @Override + public void configure(final Map<String, ?> configs, final boolean isKey) { + aggregationDeserializer.configure(configs, isKey); + headersDeserializer.configure(configs, isKey); + } + + @Override + public AggregationWithHeaders<AGG> deserialize(final String topic, final byte[] aggregationWithHeaders) { + if (aggregationWithHeaders == null) { + return null; + } + + final ByteBuffer buffer = ByteBuffer.wrap(aggregationWithHeaders); + final int headersSize = ByteUtils.readVarint(buffer); + + final byte[] rawHeaders = readBytes(buffer, headersSize); + final Headers headers = headersDeserializer.deserialize(topic, rawHeaders); + final byte[] rawAggregation = readBytes(buffer, buffer.remaining()); + final AGG aggregation = aggregationDeserializer.deserialize(topic, headers, rawAggregation); + + return AggregationWithHeaders.make(aggregation, headers); Review Comment: here if `aggregation` is null, then `make` returns null. I think this should not be the desired behaviour. Same for `ValueTimestampDeserializer.deserialize`. WDYT @frankvicky? ########## streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersDeserializer.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.utils.ByteUtils; +import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.state.AggregationWithHeaders; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer; + +/** + * Deserializer for AggregationWithHeaders. + * Deserialization format (per KIP-1271): + * [headersSize(varint)][headersBytes][aggregation] + * <p> + * 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. + * - aggregation: Serialized aggregation to be deserialized with the provided aggregation deserializer + * <p> + * This is used by KIP-1271 to deserialize aggregations with headers from session state stores. + */ +class AggregationWithHeadersDeserializer<AGG> implements WrappingNullableDeserializer<AggregationWithHeaders<AGG>, Void, AGG> { + private static final HeadersDeserializer HEADERS_DESERIALIZER = new HeadersDeserializer(); + + public final Deserializer<AGG> aggregationDeserializer; + private final HeadersDeserializer headersDeserializer; + + AggregationWithHeadersDeserializer(final Deserializer<AGG> aggregationDeserializer) { + Objects.requireNonNull(aggregationDeserializer); + this.aggregationDeserializer = aggregationDeserializer; + this.headersDeserializer = new HeadersDeserializer(); + } + + @Override + public void configure(final Map<String, ?> configs, final boolean isKey) { + aggregationDeserializer.configure(configs, isKey); + headersDeserializer.configure(configs, isKey); + } + + @Override + public AggregationWithHeaders<AGG> deserialize(final String topic, final byte[] aggregationWithHeaders) { + if (aggregationWithHeaders == null) { + return null; + } + + final ByteBuffer buffer = ByteBuffer.wrap(aggregationWithHeaders); + final int headersSize = ByteUtils.readVarint(buffer); + + final byte[] rawHeaders = readBytes(buffer, headersSize); + final Headers headers = headersDeserializer.deserialize(topic, rawHeaders); + final byte[] rawAggregation = readBytes(buffer, buffer.remaining()); + final AGG aggregation = aggregationDeserializer.deserialize(topic, headers, rawAggregation); + + return AggregationWithHeaders.make(aggregation, headers); + } + + @Override + public void close() { + aggregationDeserializer.close(); + headersDeserializer.close(); + } + + @Override + public void setIfUnset(final SerdeGetter getter) { + initNullableDeserializer(aggregationDeserializer, 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 or length is negative + */ + private static byte[] readBytes(final ByteBuffer buffer, final int length) { + if (length < 0) { + throw new SerializationException( + "Invalid AggregationWithHeaders format: negative length " + length + ); + } + if (buffer.remaining() < length) { + throw new SerializationException( + "Invalid AggregationWithHeaders format: expected " + length + + " bytes but only " + buffer.remaining() + " bytes remaining" + ); + } + final byte[] bytes = new byte[length]; + buffer.get(bytes); + return bytes; + } + + /** + * Extract aggregation from serialized AggregationWithHeaders. + */ + static <T> T aggregation(final byte[] rawAggregationWithHeaders, final Deserializer<T> deserializer) { + if (rawAggregationWithHeaders == null) { Review Comment: Not sure why we need this method, but we can go ahead and remove it if it's not needed. ########## streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersDeserializer.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.utils.ByteUtils; +import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.state.AggregationWithHeaders; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer; + +/** + * Deserializer for AggregationWithHeaders. + * Deserialization format (per KIP-1271): + * [headersSize(varint)][headersBytes][aggregation] + * <p> + * 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. + * - aggregation: Serialized aggregation to be deserialized with the provided aggregation deserializer + * <p> + * This is used by KIP-1271 to deserialize aggregations with headers from session state stores. + */ +class AggregationWithHeadersDeserializer<AGG> implements WrappingNullableDeserializer<AggregationWithHeaders<AGG>, Void, AGG> { + private static final HeadersDeserializer HEADERS_DESERIALIZER = new HeadersDeserializer(); + + public final Deserializer<AGG> aggregationDeserializer; + private final HeadersDeserializer headersDeserializer; + + AggregationWithHeadersDeserializer(final Deserializer<AGG> aggregationDeserializer) { + Objects.requireNonNull(aggregationDeserializer); + this.aggregationDeserializer = aggregationDeserializer; + this.headersDeserializer = new HeadersDeserializer(); + } + + @Override + public void configure(final Map<String, ?> configs, final boolean isKey) { + aggregationDeserializer.configure(configs, isKey); + headersDeserializer.configure(configs, isKey); + } + + @Override + public AggregationWithHeaders<AGG> deserialize(final String topic, final byte[] aggregationWithHeaders) { + if (aggregationWithHeaders == null) { + return null; + } + + final ByteBuffer buffer = ByteBuffer.wrap(aggregationWithHeaders); + final int headersSize = ByteUtils.readVarint(buffer); + + final byte[] rawHeaders = readBytes(buffer, headersSize); + final Headers headers = headersDeserializer.deserialize(topic, rawHeaders); + final byte[] rawAggregation = readBytes(buffer, buffer.remaining()); + final AGG aggregation = aggregationDeserializer.deserialize(topic, headers, rawAggregation); + + return AggregationWithHeaders.make(aggregation, headers); + } + + @Override + public void close() { + aggregationDeserializer.close(); + headersDeserializer.close(); + } + + @Override + public void setIfUnset(final SerdeGetter getter) { + initNullableDeserializer(aggregationDeserializer, 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 or length is negative + */ + private static byte[] readBytes(final ByteBuffer buffer, final int length) { + if (length < 0) { + throw new SerializationException( + "Invalid AggregationWithHeaders format: negative length " + length + ); + } + if (buffer.remaining() < length) { + throw new SerializationException( + "Invalid AggregationWithHeaders format: expected " + length + + " bytes but only " + buffer.remaining() + " bytes remaining" + ); + } + final byte[] bytes = new byte[length]; + buffer.get(bytes); + return bytes; + } + + /** + * Extract aggregation from serialized AggregationWithHeaders. + */ + static <T> T aggregation(final byte[] rawAggregationWithHeaders, final Deserializer<T> deserializer) { + if (rawAggregationWithHeaders == null) { + return null; + } + + final ByteBuffer buffer = ByteBuffer.wrap(rawAggregationWithHeaders); + final int headersSize = ByteUtils.readVarint(buffer); + // skip headers + buffer.position(buffer.position() + headersSize); + final byte[] bytes = readBytes(buffer, buffer.remaining()); + + return deserializer.deserialize("", bytes); + } + + /** + * Extract headers from serialized AggregationWithHeaders. + */ + static Headers headers(final byte[] rawAggregationWithHeaders) { + if (rawAggregationWithHeaders == null) { + return null; + } + + final ByteBuffer buffer = ByteBuffer.wrap(rawAggregationWithHeaders); + final int headersSize = ByteUtils.readVarint(buffer); + final byte[] rawHeaders = readBytes(buffer, headersSize); + return HEADERS_DESERIALIZER.deserialize("", rawHeaders); Review Comment: If you rebase the PR, you dont need the first input (`""`) any more. -- 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]
