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

Large diffs are not rendered by default.

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.SessionStoreWithHeadersBuilder;
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 @@ -720,4 +722,41 @@ public static <K, V> StoreBuilder<SessionStoreWithHeaders<K, V>> sessionStoreWit
return new SessionStoreWithHeadersBuilder<>(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);
}
Comment on lines +37 to +39
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.

why?


/**
* 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);
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.

why do add this method? for other store types we support existing api and value brings headers under the hood

}
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.

Why we changed original record instead of creating new one. see AggregationWithHeaders / ValueTimestampHeaders


/**
* 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
@@ -0,0 +1,128 @@
/*
* 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.utils.Bytes;
import org.apache.kafka.common.utils.internals.ByteUtils;
import org.apache.kafka.streams.state.HeadersBytesStore;
import org.apache.kafka.streams.state.VersionedKeyValueStoreWithHeaders;
import org.apache.kafka.streams.state.VersionedRecord;

import java.nio.ByteBuffer;
import java.util.Objects;

/**
* A persistent, versioned key-value store based on RocksDB that additionally
* preserves record headers.
* <p>
* Headers are embedded into the value bytes using the format:
* {@code [headersSize(varint)][headersBytes][rawValue]}
* before delegating to the parent {@link RocksDBVersionedStore}. On reads,
* headers are extracted from the stored value bytes.
*/
public class RocksDBVersionedStoreWithHeaders
extends RocksDBVersionedStore
implements VersionedKeyValueStoreWithHeaders<Bytes, byte[]> {

RocksDBVersionedStoreWithHeaders(final String name,
final String metricsScope,
final long historyRetention,
final long segmentInterval) {
super(name, metricsScope, historyRetention, segmentInterval);
}

@Override
public long put(final Bytes key, final byte[] value, final long timestamp, final Headers headers) {
Objects.requireNonNull(headers, "headers cannot be null");
if (value == null) {
// tombstone: delegate directly, no headers to embed
return super.put(key, null, timestamp);
}

// Check if headers are empty and use fast path
if (!headers.iterator().hasNext()) {
final byte[] encodedValue = HeadersBytesStore.convertToHeaderFormat(value);
return super.put(key, encodedValue, timestamp);
}

// Use shared HeadersSerializer infrastructure
final HeadersSerializer.PreSerializedHeaders prep = HeadersSerializer.prepareSerialization(headers);
final int payloadSize = prep.requiredBufferSizeForHeaders + value.length;
final ByteBuffer buffer = ByteBuffer.allocate(ByteUtils.sizeOfVarint(prep.requiredBufferSizeForHeaders) + payloadSize);
ByteUtils.writeVarint(prep.requiredBufferSizeForHeaders, buffer);
HeadersSerializer.serialize(prep, buffer);
buffer.put(value);
return super.put(key, buffer.array(), timestamp);
}

@Override
public long put(final Bytes key, final byte[] value, final long timestamp) {
// non-headers put: embed empty headers
return put(key, value, timestamp, new RecordHeaders());
}

@Override
public VersionedRecord<byte[]> get(final Bytes key) {
final VersionedRecord<byte[]> record = super.get(key);
return decodeRecord(record);
}

@Override
public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
final VersionedRecord<byte[]> record = super.get(key, asOfTimestamp);
return decodeRecord(record);
}

@Override
public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
final VersionedRecord<byte[]> record = super.delete(key, timestamp);
return decodeRecord(record);
}

private static VersionedRecord<byte[]> decodeRecord(final VersionedRecord<byte[]> record) {
if (record == null) {
return null;
}
final byte[] encodedValue = record.value();
final Headers headers = Utils.headers(encodedValue);
final byte[] rawValue = extractRawValue(encodedValue);
if (record.validTo().isPresent()) {
return new VersionedRecord<>(rawValue, record.timestamp(), record.validTo().get(), headers);
} else {
return new VersionedRecord<>(rawValue, record.timestamp(), headers);
}
}

/**
* Extract raw value from encoded value bytes, stripping the headers prefix.
* Format: [headersSize(varint)][headersBytes][rawValue]
*/
private static byte[] extractRawValue(final byte[] encodedValue) {
if (encodedValue == null) {
return null;
}
final ByteBuffer buffer = ByteBuffer.wrap(encodedValue);
final int headersSize = ByteUtils.readVarint(buffer);
buffer.position(buffer.position() + headersSize);
final byte[] rawValue = new byte[buffer.remaining()];
buffer.get(rawValue);
return rawValue;
}

}
Loading