Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.kafka.streams.state.VersionedBytesStoreSupplier;
import org.apache.kafka.streams.state.internals.TimestampedKeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.VersionedKeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.VersionedKeyValueStoreBuilderWithHeaders;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -51,7 +52,12 @@ public StoreBuilder<?> builder() {
: (KeyValueBytesStoreSupplier) materialized.storeSupplier();

final StoreBuilder<?> builder;
if (supplier instanceof VersionedBytesStoreSupplier) {
if (supplier instanceof VersionedBytesStoreSupplier && supplier instanceof HeadersBytesStoreSupplier) {
builder = Stores.versionedKeyValueStoreBuilderWithHeaders(
(VersionedBytesStoreSupplier) supplier,
materialized.keySerde(),
materialized.valueSerde());
} else if (supplier instanceof VersionedBytesStoreSupplier) {
builder = Stores.versionedKeyValueStoreBuilder(
(VersionedBytesStoreSupplier) supplier,
materialized.keySerde(),
Expand All @@ -75,7 +81,7 @@ public StoreBuilder<?> builder() {
}

if (materialized.cachingEnabled()) {
if (builder instanceof VersionedKeyValueStoreBuilder) {
if (builder instanceof VersionedKeyValueStoreBuilder || builder instanceof VersionedKeyValueStoreBuilderWithHeaders) {
LOG.info("Not enabling caching for store '{}' as versioned stores do not support caching.", supplier.name());
} else {
builder.withCachingEnabled();
Expand Down
39 changes: 39 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 @@ -29,6 +29,7 @@
import org.apache.kafka.streams.state.internals.RocksDbSessionBytesStoreSupplier;
import org.apache.kafka.streams.state.internals.RocksDbSessionHeadersBytesStoreSupplier;
import org.apache.kafka.streams.state.internals.RocksDbVersionedKeyValueBytesStoreSupplier;
import org.apache.kafka.streams.state.internals.RocksDbVersionedKeyValueBytesStoreWithHeadersSupplier;
import org.apache.kafka.streams.state.internals.RocksDbWindowBytesStoreSupplier;
import org.apache.kafka.streams.state.internals.SessionStoreBuilder;
import org.apache.kafka.streams.state.internals.SessionStoreBuilderWithHeaders;
Expand All @@ -37,6 +38,7 @@
import org.apache.kafka.streams.state.internals.TimestampedWindowStoreBuilder;
import org.apache.kafka.streams.state.internals.TimestampedWindowStoreWithHeadersBuilder;
import org.apache.kafka.streams.state.internals.VersionedKeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.VersionedKeyValueStoreBuilderWithHeaders;
import org.apache.kafka.streams.state.internals.WindowStoreBuilder;

import java.time.Duration;
Expand Down Expand Up @@ -683,4 +685,41 @@ public static <K, V> StoreBuilder<TimestampedKeyValueStoreWithHeaders<K, V>> tim
Objects.requireNonNull(supplier, "supplier cannot be null");
return new TimestampedKeyValueStoreBuilderWithHeaders<>(supplier, keySerde, valueSerde, Time.SYSTEM);
}

/**
* Creates a {@link VersionedBytesStoreSupplier} that also supports headers, backed by RocksDB.
*
* @param name name of the store (cannot be {@code null})
* @param historyRetention length of time that old record versions are available for query
* @return an instance of {@link VersionedBytesStoreSupplier} that also implements {@link HeadersBytesStoreSupplier}
*/
public static VersionedBytesStoreSupplier persistentVersionedKeyValueStoreWithHeaders(final String name,
final Duration historyRetention) {
Objects.requireNonNull(name, "name cannot be null");
final String msgPrefix = prepareMillisCheckFailMsgPrefix(historyRetention, "historyRetention");
final long historyRetentionMs = validateMillisecondDuration(historyRetention, msgPrefix);
if (historyRetentionMs < 0L) {
throw new IllegalArgumentException("historyRetention cannot be negative");
}
return new RocksDbVersionedKeyValueBytesStoreWithHeadersSupplier(name, historyRetentionMs);
}

/**
* Creates a {@link StoreBuilder} that can be used to build a {@link VersionedKeyValueStore}
* with headers support.
*
* @param supplier a {@link VersionedBytesStoreSupplier} (cannot be {@code null})
* @param keySerde the key serde to use
* @param valueSerde the value serde to use
* @param <K> key type
* @param <V> value type
* @return an instance of a {@link StoreBuilder} that can build a {@link VersionedKeyValueStore}
*/
public static <K, V> StoreBuilder<VersionedKeyValueStore<K, V>> versionedKeyValueStoreBuilderWithHeaders(
final VersionedBytesStoreSupplier supplier,
final Serde<K> keySerde,
final Serde<V> valueSerde) {
Objects.requireNonNull(supplier, "supplier cannot be null");
return new VersionedKeyValueStoreBuilderWithHeaders<>(supplier, keySerde, valueSerde, Time.SYSTEM);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.state;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.utils.Bytes;

/**
Expand All @@ -29,6 +30,14 @@ public interface VersionedBytesStore extends KeyValueStore<Bytes, byte[]>, Times
*/
long put(Bytes key, byte[] value, long timestamp);

/**
* The analog of {@link VersionedKeyValueStore#put(Object, Object, long)} with headers.
* Default implementation discards headers for backward compatibility.
*/
default long put(final Bytes key, final byte[] value, final long timestamp, final Headers headers) {
return put(key, value, timestamp);
}

/**
* The analog of {@link VersionedKeyValueStore#get(Object, long)}.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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.header.Headers;

/**
* A versioned key-value store that additionally preserves record headers.
* <p>
* This interface extends {@link VersionedKeyValueStore} with a headers-aware
* {@link #put(Object, Object, long, Headers)} method. On reads, headers are
* available via {@link VersionedRecord#headers()}.
*
* @param <K> The key type
* @param <V> The value type
*/
public interface VersionedKeyValueStoreWithHeaders<K, V> extends VersionedKeyValueStore<K, V> {

/**
* Add a new record version associated with the specified key, timestamp, and headers.
*
* @param key The key
* @param value The value, it can be {@code null}. {@code null} is interpreted as a delete.
* @param timestamp The timestamp for this record version
* @param headers The record headers; must not be {@code null}
* @return The validTo timestamp of the newly put record, or special values as defined
* in {@link VersionedKeyValueStore#put(Object, Object, long)}.
* @throws NullPointerException if {@code headers} is {@code null}
*/
long put(K key, V value, long timestamp, Headers headers);
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,9 @@
*/
package org.apache.kafka.streams.state;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;

import java.util.Objects;
import java.util.Optional;

Expand All @@ -29,6 +32,7 @@ public final class VersionedRecord<V> {
private final V value;
private final long timestamp;
private final Optional<Long> validTo;
private final Headers headers;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Shouldn't the equals() method also check headers equality?

Copy link
Copy Markdown
Author

@Shekharrajak Shekharrajak Apr 4, 2026

Choose a reason for hiding this comment

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

Thanks for review.

This was an intentional design choice to preserve backward compatibility. VersionedRecord.equals() is a pre-existing contract -- adding headers to it would change comparison semantics for existing code that relies on the current behavior. Headers are metadata, not part of the record version identity (two records with the same value/timestamp but different tracing headers are the same logical version). That said, I can see the argument for consistency with ValueTimestampHeaders.equals(). If the consensus is to include headers in equality, I can make that change -- though it would need a note in the doc about the behavioral change. What do you think?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

IMO headers should be part of equals and hashCode for consistency.
For backwards compatibility, new RecordHeaders() will be empty I guess, so behaviour of equals method and hashcode is still ok.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

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

You are right. For consistency with ValueTimestampHeaders.equals() and the general Java convention that all significant fields participate in equality, I will include headers in both equals() and hashCode(). The backward compatibility risk is low since existing code constructs VersionedRecord with the no-headers constructors (which default to empty RecordHeaders), and non-headers stores also return records with empty headers. I will update this in the next push.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

For consistency with ValueTimestampHeaders.equals() and the general Java convention that all significant fields participate in equality

Does it really written by you?


/**
* Create a new {@link VersionedRecord} instance. {@code value} cannot be {@code null}.
Expand All @@ -37,9 +41,7 @@ public final class VersionedRecord<V> {
* @param timestamp The type of the result returned by this query.
*/
public VersionedRecord(final V value, final long timestamp) {
this.value = Objects.requireNonNull(value, "value cannot be null.");
this.timestamp = timestamp;
this.validTo = Optional.empty();
this(value, timestamp, Optional.empty(), new RecordHeaders());
}

/**
Expand All @@ -50,11 +52,38 @@ public VersionedRecord(final V value, final long timestamp) {
* @param validTo The exclusive upper bound of the validity interval
*/
public VersionedRecord(final V value, final long timestamp, final long validTo) {
this.value = Objects.requireNonNull(value);
this.timestamp = timestamp;
this.validTo = Optional.of(validTo);
this(value, timestamp, Optional.of(validTo), new RecordHeaders());
}

/**
* Create a new {@link VersionedRecord} instance with headers. {@code value} cannot be {@code null}.
*
* @param value The value
* @param timestamp The timestamp
* @param headers The record headers
*/
public VersionedRecord(final V value, final long timestamp, final Headers headers) {
this(value, timestamp, Optional.empty(), headers);
}

/**
* Create a new {@link VersionedRecord} instance with headers. {@code value} cannot be {@code null}.
*
* @param value The value
* @param timestamp The timestamp
* @param validTo The exclusive upper bound of the validity interval
* @param headers The record headers
*/
public VersionedRecord(final V value, final long timestamp, final long validTo, final Headers headers) {
this(value, timestamp, Optional.of(validTo), headers);
}

private VersionedRecord(final V value, final long timestamp, final Optional<Long> validTo, final Headers headers) {
this.value = Objects.requireNonNull(value, "value cannot be null.");
this.timestamp = timestamp;
this.validTo = validTo;
this.headers = Objects.requireNonNull(headers, "headers cannot be null.");
}

public V value() {
return value;
Expand All @@ -68,9 +97,16 @@ public Optional<Long> validTo() {
return validTo;
}

/**
* @return the record headers. Never {@code null} (returns empty headers if none were set).
*/
public Headers headers() {
return headers;
}

@Override
public String toString() {
return "<" + value + "," + timestamp + "," + validTo + ">";
return "<" + value + "," + timestamp + "," + validTo + "," + headers + ">";
}

@Override
Expand All @@ -83,11 +119,12 @@ public boolean equals(final Object o) {
}
final VersionedRecord<?> that = (VersionedRecord<?>) o;
return timestamp == that.timestamp && validTo == that.validTo &&
Objects.equals(value, that.value);
Objects.equals(value, that.value) &&
Objects.equals(headers, that.headers);
}

@Override
public int hashCode() {
return Objects.hash(value, timestamp, validTo);
return Objects.hash(value, timestamp, validTo, headers);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,14 @@ public long put(final Bytes key, final byte[] value, final long timestamp) {
return validTo;
}

@Override
public long put(final Bytes key, final byte[] value, final long timestamp, final Headers headers) {
final Headers nonNullHeaders = headers != null ? headers : new RecordHeaders();
final long validTo = inner.put(key, value, timestamp, nonNullHeaders);
log(key, value, timestamp, nonNullHeaders);
return validTo;
}

@Override
public byte[] get(final Bytes key, final long asOfTimestamp) {
return inner.get(key, asOfTimestamp);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
Expand All @@ -32,6 +31,7 @@
import org.apache.kafka.streams.state.TimestampedKeyValueStoreWithHeaders;
import org.apache.kafka.streams.state.ValueTimestampHeaders;
import org.apache.kafka.streams.state.VersionedKeyValueStore;
import org.apache.kafka.streams.state.VersionedKeyValueStoreWithHeaders;
import org.apache.kafka.streams.state.VersionedRecord;

import java.util.Map;
Expand All @@ -51,6 +51,7 @@ public class KeyValueStoreWrapper<K, V> implements StateStore {

private TimestampedKeyValueStoreWithHeaders<K, V> headersStore = null;
private VersionedKeyValueStore<K, V> versionedStore = null;
private VersionedKeyValueStoreWithHeaders<K, V> versionedStoreWithHeaders = null;

// same as either timestampedStore or versionedStore above. kept merely as a convenience
// to simplify implementation for methods which do not depend on store type.
Expand Down Expand Up @@ -79,6 +80,9 @@ public KeyValueStoreWrapper(final ProcessorContext<?, ?> context, final String s
try {
versionedStore = context.getStateStore(storeName);
store = versionedStore;
if (versionedStore instanceof VersionedKeyValueStoreWithHeaders) {
versionedStoreWithHeaders = (VersionedKeyValueStoreWithHeaders<K, V>) versionedStore;
}
} catch (final ClassCastException e) {
store = context.getStateStore(storeName);
final String storeType = store == null ? "null" : store.getClass().getName();
Expand All @@ -93,9 +97,13 @@ public ValueTimestampHeaders<V> get(final K key) {
}
if (versionedStore != null) {
final VersionedRecord<V> versionedRecord = versionedStore.get(key);
return versionedRecord == null
? null
: ValueTimestampHeaders.make(versionedRecord.value(), versionedRecord.timestamp(), new RecordHeaders());
if (versionedRecord == null) {
return null;
}
return ValueTimestampHeaders.make(
versionedRecord.value(),
versionedRecord.timestamp(),
versionedRecord.headers());
}
throw new IllegalStateException("KeyValueStoreWrapper must be initialized with either headers or versioned store");
}
Expand All @@ -105,7 +113,13 @@ public ValueTimestampHeaders<V> get(final K key, final long asOfTimestamp) {
throw new UnsupportedOperationException("get(key, timestamp) is only supported for versioned stores");
}
final VersionedRecord<V> versionedRecord = versionedStore.get(key, asOfTimestamp);
return versionedRecord == null ? null : ValueTimestampHeaders.make(versionedRecord.value(), versionedRecord.timestamp(), new RecordHeaders());
if (versionedRecord == null) {
return null;
}
return ValueTimestampHeaders.make(
versionedRecord.value(),
versionedRecord.timestamp(),
versionedRecord.headers());
}

/**
Expand All @@ -119,6 +133,9 @@ public long put(final K key, final V value, final long timestamp, final Headers
return PUT_RETURN_CODE_IS_LATEST;
}
if (versionedStore != null) {
if (versionedStoreWithHeaders != null) {
return versionedStoreWithHeaders.put(key, value, timestamp, headers);
}
return versionedStore.put(key, value, timestamp);
}
throw new IllegalStateException("KeyValueStoreWrapper must be initialized with either headers or versioned store");
Expand Down
Loading