-
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
Changes from 3 commits
aa477af
4f1e183
0d2b7e5
44fdee1
0d1952e
fa2f7c5
f2883bf
a4a5f52
c7f7458
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,14 +16,11 @@ | |
| */ | ||
| package org.apache.kafka.streams.state.internals; | ||
|
|
||
| import org.apache.kafka.common.errors.SerializationException; | ||
| import org.apache.kafka.common.header.Header; | ||
| import org.apache.kafka.common.header.Headers; | ||
| import org.apache.kafka.common.utils.ByteUtils; | ||
|
|
||
| import java.io.ByteArrayOutputStream; | ||
| import java.io.DataOutputStream; | ||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.nio.charset.StandardCharsets; | ||
|
|
||
| /** | ||
|
|
@@ -49,49 +46,65 @@ | |
| class HeadersSerializer { | ||
|
|
||
| /** | ||
| * Serializes headers into a byte array using varint encoding per KIP-1271. | ||
| * Serializes headers into a ByteBuffer using varint encoding per KIP-1271. | ||
| * <p> | ||
| * The output format is [count][header1][header2]... without a size prefix. | ||
| * The size prefix is added by the outer serializer that uses this. | ||
| * <p> | ||
| * For null or empty headers, returns an empty byte array (0 bytes) | ||
| * instead of encoding headerCount=0 (1 byte). | ||
| * <p> | ||
| * The returned ByteBuffer may have larger capacity than actual payload size. | ||
| * It's position will be set to zero, and its limit marks the payload end. | ||
| * | ||
| * @param headers the headers to serialize (can be null) | ||
| * @return the serialized byte array (empty array if headers are null or empty) | ||
| */ | ||
| public static byte[] serialize(final Headers headers) { | ||
| public static ByteBuffer serialize(final Headers headers) { | ||
| final Header[] headersArray = (headers == null) ? new Header[0] : headers.toArray(); | ||
|
|
||
| if (headersArray.length == 0) { | ||
| return new byte[0]; | ||
| return ByteBuffer.allocate(0); | ||
| } | ||
|
|
||
| try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); | ||
| final DataOutputStream out = new DataOutputStream(baos)) { | ||
| // we first estimate an upper bound for the buffer we need, | ||
| // so we can allocate the whole buffer at once | ||
|
|
||
| ByteUtils.writeVarint(headersArray.length, out); | ||
| // start with 5 bytes for varint encoding of header count | ||
| int estimatedBufferSize = 5; | ||
| for (final Header header : headersArray) { | ||
| // adding 5 bytes for varint encoding of header-key length | ||
| estimatedBufferSize += 5 + header.key().length(); | ||
|
||
|
|
||
| for (final Header header : headersArray) { | ||
| final byte[] keyBytes = header.key().getBytes(StandardCharsets.UTF_8); | ||
| final byte[] valueBytes = header.value(); | ||
| final byte[] value = header.value(); | ||
| if (value == null) { | ||
| ++estimatedBufferSize; | ||
| } else { | ||
| // adding 5 bytes for varint encoding of header-value length | ||
| estimatedBufferSize += 5 + value.length; | ||
| } | ||
| } | ||
|
|
||
| ByteUtils.writeVarint(keyBytes.length, out); | ||
| out.write(keyBytes); | ||
| final ByteBuffer result = ByteBuffer.allocate(estimatedBufferSize); | ||
| ByteUtils.writeVarint(headersArray.length, result); | ||
|
|
||
| // Write value length and value bytes (varint + raw bytes) | ||
| // null is represented as -1, encoded as varint | ||
| if (valueBytes == null) { | ||
| ByteUtils.writeVarint(-1, out); | ||
| } else { | ||
| ByteUtils.writeVarint(valueBytes.length, out); | ||
| out.write(valueBytes); | ||
| } | ||
| } | ||
| for (final Header header : headersArray) { | ||
| final String headerKey = header.key(); | ||
| ByteUtils.writeVarint(headerKey.length(), result); | ||
| result.put(headerKey.getBytes(StandardCharsets.UTF_8)); | ||
|
|
||
| return baos.toByteArray(); | ||
| } catch (final IOException e) { | ||
| throw new SerializationException("Failed to serialize headers", e); | ||
| final byte[] headerValue = header.value(); | ||
| if (headerValue != null) { | ||
| ByteUtils.writeVarint(headerValue.length, result); | ||
| result.put(headerValue); | ||
| } else { | ||
| result.put((byte) 0x01); // hardcoded varint encoding for `-1` | ||
| } | ||
| } | ||
|
|
||
| result.limit(result.position()); | ||
| result.position(0); | ||
|
|
||
| return result; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,14 +17,8 @@ | |
| 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 +27,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 +39,9 @@ public final class RecordConverters { | |
| timestamp, | ||
| record.timestampType(), | ||
| record.serializedKeySize(), | ||
| record.serializedValueSize(), | ||
| recordValueWithTimestamp != null ? recordValueWithTimestamp.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. Stumbled over this by chance -- it's a long standing minor bug... Fixing on the side. |
||
| record.key(), | ||
| recordValue, | ||
| recordValueWithTimestamp, | ||
| record.headers(), | ||
| record.leaderEpoch() | ||
| ); | ||
|
|
@@ -57,7 +51,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 +64,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 +80,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 +92,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 +127,12 @@ 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)) { | ||
|
|
||
| ByteUtils.writeVarint(rawHeaders.length, out); | ||
| out.write(rawHeaders); | ||
| out.write(rawValue); | ||
|
|
||
| return baos.toByteArray(); | ||
| } catch (final IOException e) { | ||
| throw new SerializationException("Failed to reconstruct AggregationWithHeaders", e); | ||
| } | ||
| final ByteBuffer rawHeaders = HeadersSerializer.serialize(headers); | ||
| return Utils.prepareByteBufferWithSizePrefix(rawHeaders.limit(), rawHeaders.limit() + rawValue.length) | ||
| .put(rawHeaders) | ||
| .put(rawValue) | ||
| .array(); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -161,23 +148,12 @@ 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)) { | ||
|
|
||
| ByteUtils.writeVarint(rawHeaders.length, out); | ||
| out.write(rawHeaders); | ||
| out.write(rawTimestamp); | ||
| out.write(rawValue); | ||
|
|
||
| return baos.toByteArray(); | ||
| } catch (final IOException e) { | ||
| throw new SerializationException("Failed to reconstruct ValueTimestampHeaders", e); | ||
| } | ||
| final ByteBuffer rawHeaders = HeadersSerializer.serialize(headers); | ||
| return Utils.prepareByteBufferWithSizePrefix(rawHeaders.limit(), rawHeaders.limit() + 8 + rawValue.length) | ||
| .put(rawHeaders) | ||
| .putLong(timestamp) | ||
| .put(rawValue) | ||
| .array(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,23 @@ | |
| import java.nio.ByteBuffer; | ||
|
|
||
| public class Utils { | ||
| /** | ||
| * Create a new ByteBuffer to store {@code bufferSize} bytes, after a var-length encoded prefix. | ||
| */ | ||
| static ByteBuffer prepareByteBufferWithSizePrefix(final int prefix, final int bufferSize) { | ||
| final ByteBuffer varLengthBuffer = ByteBuffer.allocate(5); // 5 bytes for max varint encoding | ||
| ByteUtils.writeVarint(prefix, varLengthBuffer); | ||
|
||
|
|
||
| // allocate buffer with _exact_ size | ||
| final ByteBuffer buffer = ByteBuffer.allocate(varLengthBuffer.position() + bufferSize); | ||
|
|
||
| varLengthBuffer.limit(varLengthBuffer.position()); | ||
| varLengthBuffer.position(0); | ||
| buffer.put(varLengthBuffer); | ||
|
|
||
| return buffer; | ||
| } | ||
|
|
||
| /** | ||
| * Extract raw plain value from serialized ValueTimestampHeaders. | ||
| * This strips both the headers and timestamp portions. | ||
|
|
||
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.
Maybe we could use
sizeOfVarintto get exact size?