Skip to content
Draft
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
Expand Up @@ -25,8 +25,10 @@
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.SessionStore;
import org.apache.kafka.streams.state.TimestampedKeyValueStore;
import org.apache.kafka.streams.state.TimestampedKeyValueStoreWithHeaders;
import org.apache.kafka.streams.state.TimestampedWindowStore;
import org.apache.kafka.streams.state.ValueAndTimestamp;
import org.apache.kafka.streams.state.ValueTimestampHeaders;
import org.apache.kafka.streams.state.VersionedKeyValueStore;
import org.apache.kafka.streams.state.VersionedRecord;
import org.apache.kafka.streams.state.WindowStore;
Expand Down Expand Up @@ -54,7 +56,9 @@ public void close() {
}

static StateStore wrapWithReadWriteStore(final StateStore store) {
if (store instanceof TimestampedKeyValueStore) {
if (store instanceof TimestampedKeyValueStoreWithHeaders) {
return new TimestampedKeyValueStoreReadWriteDecoratorWithHeaders<>((TimestampedKeyValueStoreWithHeaders<?, ?>) store);
} else if (store instanceof TimestampedKeyValueStore) {
return new TimestampedKeyValueStoreReadWriteDecorator<>((TimestampedKeyValueStore<?, ?>) store);
} else if (store instanceof VersionedKeyValueStore) {
return new VersionedKeyValueStoreReadWriteDecorator<>((VersionedKeyValueStore<?, ?>) store);
Expand Down Expand Up @@ -149,6 +153,15 @@ static class TimestampedKeyValueStoreReadWriteDecorator<K, V>
}
}

static class TimestampedKeyValueStoreReadWriteDecoratorWithHeaders<K, V>
extends KeyValueStoreReadWriteDecorator<K, ValueTimestampHeaders<V>>
implements TimestampedKeyValueStoreWithHeaders<K, V> {

TimestampedKeyValueStoreReadWriteDecoratorWithHeaders(final TimestampedKeyValueStoreWithHeaders<K, V> inner) {
super(inner);
}
}

static class VersionedKeyValueStoreReadWriteDecorator<K, V>
extends AbstractReadWriteDecorator<VersionedKeyValueStore<K, V>, K, V>
implements VersionedKeyValueStore<K, V> {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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;

import org.apache.kafka.common.utils.ByteUtils;

import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;

/**
* Marker interface to indicate that a bytes store understands the value-with-headers format
* and can convert legacy "plain value" entries to the new format.
* <p>
* Per the KIP, the value format is: [header_length(varint)][headers_bytes][payload_bytes]
* where payload_bytes is the existing serialized value (e.g., [timestamp(8)][value] for timestamped stores).
*/
public interface HeadersBytesStore {

/**
* Converts a legacy value (without headers) to the header-embedded format.
* <p>
* For timestamped stores, the legacy format is: [timestamp(8)][value]
* The new format is: [header_length(2)][headers][timestamp(8)][value]
* <p>
* This method adds empty headers to the existing value format.
*
* @param key the key bytes (may be used for context-dependent conversion; typically unused)
* @param value the legacy value bytes (for timestamped stores: [timestamp(8)][value])
* @return the value in header-embedded format with empty headers
*/
static byte[] convertToHeaderFormat(final byte[] key, final byte[] value) {
if (value == null) {
return null;
}

try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(baos)) {

// Empty headers serialize to an empty byte array (per HeadersSerializer.serialize())
final byte[] emptyHeadersBytes = new byte[0];

// Write format: [headers_size(varint)][headers_bytes][payload]
ByteUtils.writeVarint(emptyHeadersBytes.length, out); // headers_size = 0
// No headers_bytes to write (empty array)
out.write(value); // payload: [timestamp(8)][value]

return baos.toByteArray();
} catch (IOException e) {
throw new RuntimeException("Failed to convert to header format", e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,13 @@
*/
package org.apache.kafka.streams.state;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.state.internals.ValueAndTimestampSerializer;

Expand Down Expand Up @@ -149,26 +152,49 @@ public String topic() {
* @param rawKey the key as raw bytes
* @return the key as typed object
*/
// @Deprecated
public K keyFrom(final byte[] rawKey) {
return keySerde.deserializer().deserialize(topic, rawKey);
}

/**
* Deserialize the key from raw bytes.
*
* @param rawKey the key as raw bytes
* @return the key as typed object
*/
public K keyFrom(final byte[] rawKey, final Headers headers) {
return keySerde.deserializer().deserialize(topic, headers, rawKey);
}

/**
* Deserialize the value from raw bytes.
*
* @param rawValue the value as raw bytes
* @return the value as typed object
*/
// @Deprecated
public V valueFrom(final byte[] rawValue) {
return valueSerde.deserializer().deserialize(topic, rawValue);
}

/**
* Deserialize the value from raw bytes.
*
* @param rawValue the value as raw bytes
* @return the value as typed object
*/
public V valueFrom(final byte[] rawValue, Headers headers) {
return valueSerde.deserializer().deserialize(topic, headers, Utils.wrapNullable(rawValue));
}

/**
* Serialize the given key.
*
* @param key the key to be serialized
* @return the serialized key
*/
// @Deprecated
public byte[] rawKey(final K key) {
try {
return keySerde.serializer().serialize(topic, key);
Expand All @@ -184,12 +210,34 @@ public byte[] rawKey(final K key) {
}
}

/**
* Serialize the given key.
*
* @param key the key to be serialized
* @return the serialized key
*/
public byte[] rawKey(final K key, final Headers headers) {
try {
return keySerde.serializer().serialize(topic, headers, key);
} catch (final ClassCastException e) {
final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName();
throw new StreamsException(
String.format("A serializer (%s) is not compatible to the actual key type " +
"(key type: %s). Change the default Serdes in StreamConfig or " +
"provide correct Serdes via method parameters.",
keySerializer().getClass().getName(),
keyClass),
e);
}
}

/**
* Serialize the given value.
*
* @param value the value to be serialized
* @return the serialized value
*/
// @Deprecated
@SuppressWarnings("rawtypes")
public byte[] rawValue(final V value) {
try {
Expand All @@ -213,4 +261,34 @@ public byte[] rawValue(final V value) {
e);
}
}

/**
* Serialize the given value.
*
* @param value the value to be serialized
* @return the serialized value
*/
@SuppressWarnings("rawtypes")
public byte[] rawValue(final V value, final Headers headers) {
try {
return valueSerde.serializer().serialize(topic, headers, value);
} catch (final ClassCastException e) {
final String valueClass;
final Class<? extends Serializer> serializerClass;
if (valueSerializer() instanceof ValueAndTimestampSerializer) {
serializerClass = ((ValueAndTimestampSerializer<?>) valueSerializer()).valueSerializer.getClass();
valueClass = value == null ? "unknown because value is null" : ((ValueAndTimestamp) value).value().getClass().getName();
} else {
serializerClass = valueSerializer().getClass();
valueClass = value == null ? "unknown because value is null" : value.getClass().getName();
}
throw new StreamsException(
String.format("A serializer (%s) is not compatible to the actual value type " +
"(value type: %s). Change the default Serdes in StreamConfig or " +
"provide correct Serdes via method parameters.",
serializerClass.getName(),
valueClass),
e);
}
}
}
13 changes: 13 additions & 0 deletions streams/src/main/java/org/apache/kafka/streams/state/Stores.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.kafka.streams.state.internals.RocksDbWindowBytesStoreSupplier;
import org.apache.kafka.streams.state.internals.SessionStoreBuilder;
import org.apache.kafka.streams.state.internals.TimestampedKeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.TimestampedKeyValueStoreBuilderWithHeaders;
import org.apache.kafka.streams.state.internals.TimestampedWindowStoreBuilder;
import org.apache.kafka.streams.state.internals.VersionedKeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.WindowStoreBuilder;
Expand Down Expand Up @@ -116,6 +117,11 @@ public static KeyValueBytesStoreSupplier persistentTimestampedKeyValueStore(fina
return new RocksDBKeyValueBytesStoreSupplier(name, true);
}

public static KeyValueBytesStoreSupplier persistentTimestampedKeyValueStoreWithHeaders(final String name) {
Objects.requireNonNull(name, "name cannot be null");
return new RocksDBKeyValueBytesStoreSupplier(name, true, true);
}

/**
* Create a persistent versioned key-value store {@link VersionedBytesStoreSupplier}.
* <p>
Expand Down Expand Up @@ -485,6 +491,13 @@ public static <K, V> StoreBuilder<TimestampedKeyValueStore<K, V>> timestampedKey
return new TimestampedKeyValueStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM);
}

public static <K, V> StoreBuilder<TimestampedKeyValueStoreWithHeaders<K, V>> timestampedKeyValueStoreBuilderWithHeaders(final KeyValueBytesStoreSupplier supplier,
final Serde<K> keySerde,
final Serde<V> valueSerde) {
Objects.requireNonNull(supplier, "supplier cannot be null");
return new TimestampedKeyValueStoreBuilderWithHeaders<>(supplier, keySerde, valueSerde, Time.SYSTEM);
}

/**
* Creates a {@link StoreBuilder} that can be used to build a {@link VersionedKeyValueStore}.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,11 @@
package org.apache.kafka.streams.state;

/**
* A key-(value/timestamp/headers) store that supports put/get/delete.
*
* @param <K> The key type
* @param <V> The value type
*/
public interface TimestampedKeyValueStoreWithHeaders<K, V>
extends KeyValueStore<K, ValueTimestampHeaders<V>> {
}
Loading