Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.utils.ByteUtils;

import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;

/**
* Deserializer for Kafka Headers.
*
* Deserialization format (per KIP-1271):
* [NumHeaders(varint)][Header1][Header2]...
*
* Each header:
* [KeyLength(varint)][KeyBytes(UTF-8)][ValueLength(varint)][ValueBytes]
*
* Note: ValueLength is -1 for null values (encoded as varint).
* All integers are decoded from varints (signed varint encoding).
*
* This deserializer expects the headersBytes portion without a size prefix.
* The size prefix is handled by the outer deserializer (e.g., ValueTimestampHeadersDeserializer).
*
* This is used by KIP-1271 to deserialize headers from state stores.
*/
public class HeadersDeserializer implements Deserializer<Headers> {

/**
* Deserializes headers from a byte array using varint encoding per KIP-1271.
* <p>
* The input format is [count][header1][header2]... without a size prefix.
*
* @param topic topic associated with the data
* @param data the serialized byte array (can be null)
* @return the deserialized headers
*/
public Headers deserialize(final String topic, final byte[] data) {
if (data == null || data.length == 0) {
return new RecordHeaders();
}

final ByteBuffer buffer = ByteBuffer.wrap(data);
final int headerCount = ByteUtils.readVarint(buffer);

if (headerCount == 0) {
return new RecordHeaders();
}

final RecordHeaders headers = new RecordHeaders();

for (int i = 0; i < headerCount; i++) {
final int keyLength = ByteUtils.readVarint(buffer);
final byte[] keyBytes = new byte[keyLength];
buffer.get(keyBytes);
final String key = new String(keyBytes, StandardCharsets.UTF_8);

final int valueLength = ByteUtils.readVarint(buffer);
final byte[] value;
if (valueLength == -1) {
value = null;
} else {
value = new byte[valueLength];
buffer.get(value);
}

headers.add(key, value);
}

return headers;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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.Header;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.ByteUtils;

import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;

/**
* Serializer for Kafka Headers.
* <p>
* Serialization format (per KIP-1271):
* [NumHeaders(varint)][Header1][Header2]...
* <p>
* Each header:
* [KeyLength(varint)][KeyBytes(UTF-8)][ValueLength(varint)][ValueBytes]
* <p>
* Note: ValueLength is -1 for null values (encoded as varint).
* All integers are encoded as varints (signed varint encoding).
* <p>
* This serializer produces the headersBytes portion. The headersSize prefix
* is added by the outer serializer (e.g., ValueTimestampHeadersSerializer).
* <p>
* This is used by KIP-1271 to serialize headers for storage in state stores.
*/
public class HeadersSerializer implements Serializer<Headers> {

/**
* Serializes headers into a byte array 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.
*
* @param topic topic associated with data
* @param headers the headers to serialize (can be null)
* @return the serialized byte array
*/
@Override
public byte[] serialize(final String topic, final Headers headers) {
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final DataOutputStream out = new DataOutputStream(baos)) {

final Header[] headerArray = (headers == null) ? new Header[0] : headers.toArray();
ByteUtils.writeVarint(headerArray.length, out);

for (final Header header : headerArray) {
final byte[] keyBytes = header.key().getBytes(StandardCharsets.UTF_8);
final byte[] valueBytes = header.value();

ByteUtils.writeVarint(keyBytes.length, out);
out.write(keyBytes);

// 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);
}
}

return baos.toByteArray();
} catch (IOException e) {
throw new RuntimeException("Failed to serialize headers", e);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/*
* 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.Header;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;

import org.junit.jupiter.api.Test;

import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;

public class HeadersDeserializerTest {

private final HeadersSerializer serializer = new HeadersSerializer();
private final HeadersDeserializer deserializer = new HeadersDeserializer();

@Test
public void shouldDeserializeNullData() {
final Headers headers = deserializer.deserialize("", null);

assertNotNull(headers);
assertEquals(0, headers.toArray().length);
}

@Test
public void shouldDeserializeEmptyData() {
final Headers headers = deserializer.deserialize("", new byte[0]);

assertNotNull(headers);
assertEquals(0, headers.toArray().length);
}

@Test
public void shouldRoundTripEmptyHeaders() {
final Headers original = new RecordHeaders();
final byte[] serialized = serializer.serialize("", original);
final Headers deserialized = deserializer.deserialize("", serialized);

assertNotNull(deserialized);
assertEquals(0, deserialized.toArray().length);
}

@Test
public void shouldRoundTripSingleHeader() {
final Headers original = new RecordHeaders()
.add("key1", "value1".getBytes());
final byte[] serialized = serializer.serialize("", original);
final Headers deserialized = deserializer.deserialize("", serialized);

assertNotNull(deserialized);
assertEquals(1, deserialized.toArray().length);

final Header header = deserialized.lastHeader("key1");
assertNotNull(header);
assertEquals("key1", header.key());
assertArrayEquals("value1".getBytes(), header.value());
}

@Test
public void shouldRoundTripMultipleHeaders() {
final Headers original = new RecordHeaders()
.add("key0", "value0".getBytes())
.add("key1", "value1".getBytes())
.add("key2", "value2".getBytes());
final byte[] serialized = serializer.serialize("", original);
final Headers deserialized = deserializer.deserialize("", serialized);
assertNotNull(deserialized);

final Header[] headerArray = deserialized.toArray();
assertEquals(3, headerArray.length);
for (int i = 0; i < headerArray.length; i++) {
Header next = headerArray[i];
assertEquals("key" + i, next.key());
assertArrayEquals(("value" + i).getBytes(), next.value());
}
}

@Test
public void shouldRoundTripHeaderWithNullValue() {
final Headers original = new RecordHeaders()
.add("key1", null);
final byte[] serialized = serializer.serialize("", original);
final Headers deserialized = deserializer.deserialize("", serialized);

assertNotNull(deserialized);
assertEquals(1, deserialized.toArray().length);

final Header header = deserialized.lastHeader("key1");
assertNotNull(header);
assertEquals("key1", header.key());
assertNull(header.value());
}

@Test
public void shouldRoundTripHeaderWithEmptyValue() {
final Headers original = new RecordHeaders()
.add("key1", new byte[0]);
final byte[] serialized = serializer.serialize("", original);
final Headers deserialized = deserializer.deserialize("", serialized);

assertNotNull(deserialized);
assertEquals(1, deserialized.toArray().length);

final Header header = deserialized.lastHeader("key1");
assertNotNull(header);
assertEquals("key1", header.key());
assertArrayEquals(new byte[0], header.value());
}
}
Loading