-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-20399: Implement VersionedKeyValueStoreWithHeaders #21995
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
base: trunk
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
| 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); | ||
|
Contributor
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. 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 |
|---|---|---|
|
|
@@ -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; | ||
|
|
||
|
|
@@ -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; | ||
|
Contributor
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. 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}. | ||
|
|
@@ -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()); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -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; | ||
|
|
@@ -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 | ||
|
|
@@ -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 |
|---|---|---|
| @@ -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; | ||
| } | ||
|
|
||
| } |
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.
why?