Skip to content

Comments

KAFKA-20158: Add AggregationWithHeaders, serialization support and tests (1/N)#21511

Open
bbejeck wants to merge 3 commits intoapache:trunkfrom
bbejeck:KAFKA-20158_session_store_w_headers_part_1
Open

KAFKA-20158: Add AggregationWithHeaders, serialization support and tests (1/N)#21511
bbejeck wants to merge 3 commits intoapache:trunkfrom
bbejeck:KAFKA-20158_session_store_w_headers_part_1

Conversation

@bbejeck
Copy link
Member

@bbejeck bbejeck commented Feb 18, 2026

This PR introduces AggregationWithHeaders and serialization support
introduced in KIP-1271 for storing session aggregations with headers.

@bbejeck bbejeck requested a review from frankvicky February 18, 2026 21:43
@bbejeck bbejeck changed the title KAFKA-20158: Add AggregationWithHeaders, serialization support and tests KAFKA-20158: Add AggregationWithHeaders, serialization support and tests (1/N) Feb 18, 2026
Copy link
Contributor

@aliehsaeedii aliehsaeedii left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @bbejeck. I left some minor comments.

* <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> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This class is package-private while AggregationWithHeadersSerializer is public

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe both classes need to be package-private?


return baos.toByteArray();
} catch (final IOException e) {
throw new SerializationException("Failed to serialize AggregationWithHeaders", e);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we add topic to the exception message for better debugging?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if so, we should also add it to ValueTimestampHeadersSerializer

* @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) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Member Author

@bbejeck bbejeck Feb 20, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great idea! But we currently don't have a Utils class (at least that I know of) can we defer this to a follow-up PR and maybe consider other code as well?

* @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) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks. I like this check. We dont have it in other places.

final ByteBuffer buffer = ByteBuffer.wrap(rawAggregationWithHeaders);
final int headersSize = ByteUtils.readVarint(buffer);
final byte[] rawHeaders = readBytes(buffer, headersSize);
return HEADERS_DESERIALIZER.deserialize("", rawHeaders);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you rebase the PR, you dont need the first input ("") any more.

* Extract aggregation from serialized AggregationWithHeaders.
*/
static <T> T aggregation(final byte[] rawAggregationWithHeaders, final Deserializer<T> deserializer) {
if (rawAggregationWithHeaders == null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure why we need this method, but we can go ahead and remove it if it's not needed.

final byte[] rawAggregation = readBytes(buffer, buffer.remaining());
final AGG aggregation = aggregationDeserializer.deserialize(topic, headers, rawAggregation);

return AggregationWithHeaders.make(aggregation, headers);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

here if aggregation is null, then make returns null. I think this should not be the desired behaviour. Same for ValueTimestampDeserializer.deserialize. WDYT @frankvicky?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmmm...
Make sense, even if we have a null value after deserialization, the headers might still be meaningful.
We should replace with makeAllowNullable

Copy link
Contributor

@frankvicky frankvicky left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall LGTM.
Could you please rebase on the trunk?
There are some changes of deserializer/serializer of headers.

@bbejeck bbejeck force-pushed the KAFKA-20158_session_store_w_headers_part_1 branch from 23ff16b to 3110503 Compare February 20, 2026 17:03
@bbejeck
Copy link
Member Author

bbejeck commented Feb 20, 2026

@aliehsaeedii @frankvicky all comments addressed ready for another review

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants