-
Notifications
You must be signed in to change notification settings - Fork 15.1k
MINOR: perf optimization for header serialization and type conversion #21762
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
mjsax
merged 9 commits into
apache:trunk
from
mjsax:minor-optimize-header-serializer-bytebuffer-no-bytearrayoutputstream
Mar 16, 2026
Merged
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
aa477af
MINOR: perf optimization for header serialization and type conversion
mjsax 4f1e183
cleanup
mjsax 0d2b7e5
minor
mjsax 44fdee1
review comments
mjsax 0d1952e
review comments
mjsax fa2f7c5
testing
mjsax f2883bf
avoid expensive array operations which totally kill perf
mjsax a4a5f52
checkstyle and cleanup
mjsax c7f7458
Apply suggestion from @mjsax
mjsax File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,14 +17,9 @@ | |
| package org.apache.kafka.streams.state.internals; | ||
|
|
||
| import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
| import org.apache.kafka.common.errors.SerializationException; | ||
| import org.apache.kafka.common.header.Headers; | ||
| import org.apache.kafka.common.serialization.LongSerializer; | ||
| import org.apache.kafka.common.utils.ByteUtils; | ||
|
|
||
| import java.io.ByteArrayOutputStream; | ||
| import java.io.DataOutputStream; | ||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
|
|
||
| public final class RecordConverters { | ||
|
|
@@ -33,7 +28,7 @@ public final class RecordConverters { | |
| private static final RecordConverter RAW_TO_TIMESTAMED_INSTANCE = record -> { | ||
| final byte[] rawValue = record.value(); | ||
| final long timestamp = record.timestamp(); | ||
| final byte[] recordValue = rawValue == null ? null : | ||
| final byte[] recordValueWithTimestamp = rawValue == null ? null : | ||
| ByteBuffer.allocate(8 + rawValue.length) | ||
| .putLong(timestamp) | ||
| .put(rawValue) | ||
|
|
@@ -45,9 +40,9 @@ public final class RecordConverters { | |
| timestamp, | ||
| record.timestampType(), | ||
| record.serializedKeySize(), | ||
| record.serializedValueSize(), | ||
| recordValueWithTimestamp != null ? recordValueWithTimestamp.length : 0, | ||
| record.key(), | ||
| recordValue, | ||
| recordValueWithTimestamp, | ||
| record.headers(), | ||
| record.leaderEpoch() | ||
| ); | ||
|
|
@@ -57,7 +52,7 @@ public final class RecordConverters { | |
| final byte[] rawValue = record.value(); | ||
|
|
||
| // Format: [headersSize(varint)][headersBytes][timestamp(8)][value] | ||
| final byte[] recordValue = reconstructFromRaw( | ||
| final byte[] recordValueWithTimestampAndHeaders = reconstructFromRaw( | ||
| rawValue, | ||
| record.timestamp(), | ||
| record.headers() | ||
|
|
@@ -70,9 +65,9 @@ public final class RecordConverters { | |
| record.timestamp(), | ||
| record.timestampType(), | ||
| record.serializedKeySize(), | ||
| record.serializedValueSize(), | ||
| recordValueWithTimestampAndHeaders != null ? recordValueWithTimestampAndHeaders.length : 0, | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same... |
||
| record.key(), | ||
| recordValue, | ||
| recordValueWithTimestampAndHeaders, | ||
| record.headers(), | ||
| record.leaderEpoch() | ||
| ); | ||
|
|
@@ -86,7 +81,7 @@ public static RecordConverter rawValueToHeadersValue() { | |
| final byte[] rawValue = record.value(); | ||
|
|
||
| // Format: [headersSize(varint)][headersBytes][aggregation] (no timestamp) | ||
| final byte[] recordValue = reconstructSessionFromRaw( | ||
| final byte[] recordValueWithHeaders = reconstructSessionFromRaw( | ||
| rawValue, | ||
| record.headers() | ||
| ); | ||
|
|
@@ -98,9 +93,9 @@ public static RecordConverter rawValueToHeadersValue() { | |
| record.timestamp(), | ||
| record.timestampType(), | ||
| record.serializedKeySize(), | ||
| record.serializedValueSize(), | ||
| recordValueWithHeaders != null ? recordValueWithHeaders.length : 0, | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. And one more |
||
| record.key(), | ||
| recordValue, | ||
| recordValueWithHeaders, | ||
| record.headers(), | ||
| record.leaderEpoch() | ||
| ); | ||
|
|
@@ -133,19 +128,18 @@ static byte[] reconstructSessionFromRaw(final byte[] rawValue, final Headers hea | |
| if (rawValue == null) { | ||
| return null; | ||
| } | ||
| final byte[] rawHeaders = HeadersSerializer.serialize(headers); | ||
|
|
||
| try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); | ||
| final DataOutputStream out = new DataOutputStream(baos)) { | ||
| final HeadersSerializer.PreSerializedHeaders preSerializedHeaders = HeadersSerializer.prepareSerialization(headers); | ||
|
|
||
| ByteUtils.writeVarint(rawHeaders.length, out); | ||
| out.write(rawHeaders); | ||
| out.write(rawValue); | ||
| final int payloadSize = preSerializedHeaders.requiredBufferSizeForHeaders + rawValue.length; | ||
|
|
||
| return baos.toByteArray(); | ||
| } catch (final IOException e) { | ||
| throw new SerializationException("Failed to reconstruct AggregationWithHeaders", e); | ||
| } | ||
| // Format: [headersSize(varint)][headersBytes][value] | ||
| final ByteBuffer buffer = ByteBuffer.allocate(ByteUtils.sizeOfVarint(preSerializedHeaders.requiredBufferSizeForHeaders) + payloadSize); | ||
| ByteUtils.writeVarint(preSerializedHeaders.requiredBufferSizeForHeaders, buffer); | ||
|
|
||
| return HeadersSerializer.serialize(preSerializedHeaders, buffer) | ||
| .put(rawValue) | ||
| .array(); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -161,23 +155,18 @@ static byte[] reconstructFromRaw(final byte[] rawValue, final long timestamp, fi | |
| if (rawValue == null) { | ||
| return null; | ||
| } | ||
| final byte[] rawTimestamp; | ||
| try (LongSerializer timestampSerializer = new LongSerializer()) { | ||
| rawTimestamp = timestampSerializer.serialize("", timestamp); | ||
| } | ||
| final byte[] rawHeaders = HeadersSerializer.serialize(headers); | ||
|
|
||
| try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); | ||
| final DataOutputStream out = new DataOutputStream(baos)) { | ||
| final HeadersSerializer.PreSerializedHeaders preSerializedHeaders = HeadersSerializer.prepareSerialization(headers); | ||
|
|
||
| ByteUtils.writeVarint(rawHeaders.length, out); | ||
| out.write(rawHeaders); | ||
| out.write(rawTimestamp); | ||
| out.write(rawValue); | ||
| final int payloadSize = preSerializedHeaders.requiredBufferSizeForHeaders + 8 + rawValue.length; | ||
|
|
||
| return baos.toByteArray(); | ||
| } catch (final IOException e) { | ||
| throw new SerializationException("Failed to reconstruct ValueTimestampHeaders", e); | ||
| } | ||
| // Format: [headersSize(varint)][headersBytes][timestamp(8)][value] | ||
| final ByteBuffer buffer = ByteBuffer.allocate(ByteUtils.sizeOfVarint(preSerializedHeaders.requiredBufferSizeForHeaders) + payloadSize); | ||
| ByteUtils.writeVarint(preSerializedHeaders.requiredBufferSizeForHeaders, buffer); | ||
|
|
||
| return HeadersSerializer.serialize(preSerializedHeaders, buffer) | ||
| .putLong(timestamp) | ||
| .put(rawValue) | ||
| .array(); | ||
| } | ||
| } | ||
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Stumbled over this by chance -- it's a long standing minor bug... Fixing on the side.