-
Notifications
You must be signed in to change notification settings - Fork 15.2k
KAFKA-20173: Ensure Metered session-stores pass headers correctly #21957
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
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.
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,7 +18,6 @@ | |
|
|
||
| import org.apache.kafka.common.header.Headers; | ||
| import org.apache.kafka.common.header.internals.RecordHeaders; | ||
| import org.apache.kafka.common.metrics.Sensor; | ||
| import org.apache.kafka.common.serialization.Serde; | ||
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.apache.kafka.common.utils.Time; | ||
|
|
@@ -45,9 +44,6 @@ | |
| import org.apache.kafka.streams.state.WindowStoreIterator; | ||
|
|
||
| import java.util.Objects; | ||
| import java.util.Set; | ||
| import java.util.concurrent.atomic.LongAdder; | ||
| import java.util.function.BiFunction; | ||
| import java.util.function.Function; | ||
|
|
||
| import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; | ||
|
|
@@ -190,10 +186,7 @@ private <R> QueryResult<R> runWindowKeyQuery( | |
| return vth == null ? null : ValueAndTimestamp.make(vth.value(), vth.timestamp()); | ||
| } | ||
| ); | ||
|
|
||
| final QueryResult<MeteredWindowStoreIterator<ValueAndTimestamp<V>>> typedQueryResult = | ||
| InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); | ||
| queryResult = (QueryResult<R>) typedQueryResult; | ||
| queryResult = (QueryResult<R>) InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); | ||
| } else { | ||
| // For non-timestamped stores, return plain V | ||
| final MeteredWindowStoreIterator<V> typedResult = meteredIterator( | ||
|
|
@@ -203,10 +196,7 @@ private <R> QueryResult<R> runWindowKeyQuery( | |
| return vth == null ? null : vth.value(); | ||
| } | ||
| ); | ||
|
|
||
| final QueryResult<MeteredWindowStoreIterator<V>> typedQueryResult = | ||
| InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); | ||
| queryResult = (QueryResult<R>) typedQueryResult; | ||
| queryResult = (QueryResult<R>) InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); | ||
| } | ||
| } else { | ||
| queryResult = (QueryResult<R>) rawResult; | ||
|
|
@@ -430,59 +420,16 @@ private <ValueType> MeteredWindowedKeyValueIterator<K, ValueType> meteredWindowe | |
| rawResult.getResult(), | ||
| fetchSensor, | ||
| iteratorDurationSensor, | ||
| this::deserializeValue, | ||
| this::deserializeKey, | ||
| ValueTimestampHeaders::headers, | ||
| valueConverter, | ||
| time, | ||
| numOpenIterators, | ||
| openIterators | ||
| ); | ||
| } | ||
|
|
||
| private final class MeteredWindowedKeyValueWithHeadersIterator<ValueType> extends MeteredWindowedKeyValueIterator<K, ValueType> { | ||
|
Member
Author
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. Extracted into standalone class, so we can also use it in |
||
| private final BiFunction<byte[], Headers, K> deserializeKey; | ||
| private final Function<ValueTimestampHeaders<V>, ValueType> valueConverter; | ||
|
|
||
| MeteredWindowedKeyValueWithHeadersIterator( | ||
| final KeyValueIterator<Windowed<Bytes>, byte[]> iter, | ||
| final Sensor operationSensor, | ||
| final Sensor iteratorSensor, | ||
| final BiFunction<byte[], Headers, K> deserializeKey, | ||
| final Function<ValueTimestampHeaders<V>, ValueType> valueConverter, | ||
| final Time time, | ||
| final LongAdder numOpenIterators, | ||
| final Set<MeteredIterator> openIterators | ||
| ) { | ||
| super( | ||
| iter, | ||
| operationSensor, | ||
| iteratorSensor, | ||
| null, // should not be used in super-class | ||
| null, // should not be used in super-class | ||
| time, | ||
| numOpenIterators, | ||
| openIterators | ||
| ); | ||
|
|
||
| this.deserializeKey = deserializeKey; | ||
| this.valueConverter = valueConverter; | ||
| } | ||
|
|
||
| @Override | ||
| public KeyValue<Windowed<K>, ValueType> next() { | ||
| final KeyValue<Windowed<Bytes>, byte[]> next = iter.next(); | ||
| final ValueTimestampHeaders<V> valueTimestampHeaders = deserializeValue(next.value); | ||
| return KeyValue.pair( | ||
| windowedKey(next.key, valueTimestampHeaders.headers()), | ||
| valueConverter.apply(valueTimestampHeaders) | ||
| ); | ||
| } | ||
|
|
||
| private Windowed<K> windowedKey(final Windowed<Bytes> bytesKey, final Headers headers) { | ||
| final K key = deserializeKey.apply(bytesKey.key().get(), headers); | ||
| return new Windowed<>(key, bytesKey.window()); | ||
| } | ||
| } | ||
|
|
||
| private boolean isUnderlyingStoreTimestamped() { | ||
| StateStore store = wrapped(); | ||
| do { | ||
|
|
@@ -507,16 +454,11 @@ private boolean isUnderlyingStoreTimestamped() { | |
| return false; | ||
| } | ||
|
|
||
| protected Bytes serializeKey(final K key, final Headers headers) { | ||
| private Bytes serializeKey(final K key, final Headers headers) { | ||
| return Bytes.wrap(serdes.rawKey(key, headers)); | ||
| } | ||
|
|
||
| @Override | ||
| protected K deserializeKey(final byte[] rawKey) { | ||
| throw new UnsupportedOperationException("MeteredTimestampedWindowStoreWithHeaders required to pass in Headers when deserializing a key."); | ||
|
Member
Author
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. We can make the super class method private so we don't need this override any longer.
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. +1 for this clean up |
||
| } | ||
|
|
||
| protected K deserializeKey(final byte[] rawKey, final Headers headers) { | ||
| private K deserializeKey(final byte[] rawKey, final Headers headers) { | ||
| return serdes.keyFrom(rawKey, headers); | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -541,7 +541,7 @@ private Bytes serializeKey(final K key) { | |
| return Bytes.wrap(serdes.rawKey(key, internalContext.headers())); | ||
| } | ||
|
|
||
| protected K deserializeKey(final byte[] rawKey) { | ||
| private K deserializeKey(final byte[] rawKey) { | ||
|
Member
Author
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. Side cleanup -- sub-classes don't use this anyway, so we can make it |
||
| return serdes.keyFrom(rawKey, internalContext.headers()); | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,81 @@ | ||
| /* | ||
| * 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.metrics.Sensor; | ||
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.apache.kafka.common.utils.Time; | ||
| import org.apache.kafka.streams.KeyValue; | ||
| import org.apache.kafka.streams.kstream.Windowed; | ||
| import org.apache.kafka.streams.state.KeyValueIterator; | ||
|
|
||
| import java.util.Set; | ||
| import java.util.concurrent.atomic.LongAdder; | ||
| import java.util.function.BiFunction; | ||
| import java.util.function.Function; | ||
|
|
||
| final class MeteredWindowedKeyValueWithHeadersIterator<K, VInner, VOuter> extends MeteredWindowedKeyValueIterator<K, VOuter> { | ||
| private final Function<byte[], VInner> deserializeValue; | ||
| private final BiFunction<byte[], Headers, K> deserializeKey; | ||
| private final Function<VInner, Headers> headersExtractor; | ||
|
Member
Author
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. To make this iterator generic (ie, work with |
||
| private final Function<VInner, VOuter> valueConverter; | ||
|
|
||
| MeteredWindowedKeyValueWithHeadersIterator( | ||
| final KeyValueIterator<Windowed<Bytes>, byte[]> iter, | ||
| final Sensor operationSensor, | ||
| final Sensor iteratorSensor, | ||
| final Function<byte[], VInner> deserializeValue, | ||
| final BiFunction<byte[], Headers, K> deserializeKey, | ||
| final Function<VInner, Headers> headersExtractor, | ||
| final Function<VInner, VOuter> valueConverter, | ||
| final Time time, | ||
| final LongAdder numOpenIterators, | ||
| final Set<MeteredIterator> openIterators | ||
| ) { | ||
| super( | ||
| iter, | ||
| operationSensor, | ||
| iteratorSensor, | ||
| null, // should not be used in super-class | ||
| null, // should not be used in super-class | ||
| time, | ||
| numOpenIterators, | ||
| openIterators | ||
| ); | ||
|
|
||
| this.deserializeValue = deserializeValue; | ||
| this.deserializeKey = deserializeKey; | ||
| this.headersExtractor = headersExtractor; | ||
| this.valueConverter = valueConverter; | ||
| } | ||
|
|
||
| @Override | ||
| public KeyValue<Windowed<K>, VOuter> next() { | ||
| final KeyValue<Windowed<Bytes>, byte[]> next = iter.next(); | ||
| final VInner valueTimestampHeaders = deserializeValue.apply(next.value); | ||
| return KeyValue.pair( | ||
| windowedKey(next.key, headersExtractor.apply(valueTimestampHeaders)), | ||
| valueConverter.apply(valueTimestampHeaders) | ||
| ); | ||
| } | ||
|
|
||
| private Windowed<K> windowedKey(final Windowed<Bytes> bytesKey, final Headers headers) { | ||
| final K key = deserializeKey.apply(bytesKey.key().get(), headers); | ||
| return new Windowed<>(key, bytesKey.window()); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,9 +20,7 @@ | |
| import org.apache.kafka.common.header.Headers; | ||
| import org.apache.kafka.common.header.internals.RecordHeaders; | ||
| import org.apache.kafka.common.serialization.LongDeserializer; | ||
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.apache.kafka.common.utils.internals.ByteUtils; | ||
| import org.apache.kafka.streams.kstream.Windowed; | ||
| import org.apache.kafka.streams.state.StateSerdes; | ||
|
|
||
| import java.nio.ByteBuffer; | ||
|
|
@@ -54,38 +52,6 @@ public static Headers headers(final byte[] valueWithHeaders) { | |
| return readHeaders(buffer); | ||
| } | ||
|
|
||
| /** | ||
|
Member
Author
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. We don't need these any longer -- we have the corresponding method inside the stores, and I don't think it make sense to extract them into this |
||
| * Serialize the key with headers into bytes | ||
| * @param key the key to serialize | ||
| * @param headers the Headers as context | ||
| * @param serdes the StateSerdes as serializer | ||
| * @return the Bytes of the key | ||
| */ | ||
| public static <K> Bytes keyBytes(final K key, final Headers headers, final StateSerdes<K, ?> serdes) { | ||
| return Bytes.wrap(serdes.rawKey(key, headers)); | ||
| } | ||
|
|
||
| /** | ||
| * Serialize the key into bytes | ||
| * @param key the key to serialize | ||
| * @param serdes the StateSerdes as serializer | ||
| * @return the Bytes of the key | ||
| */ | ||
| static <K> Bytes keyBytes(final K key, final StateSerdes<K, ?> serdes) { | ||
| return keyBytes(key, new RecordHeaders(), serdes); | ||
| } | ||
|
|
||
| /** | ||
| * Serialize the session key with headers into bytes | ||
| * @param sessionKey the Windowed session key to serialize | ||
| * @param headers the Headers as context | ||
| * @param serdes the StateSerdes as serializer | ||
| * @return the Bytes of the key | ||
| */ | ||
| static <K> Bytes keyBytes(final Windowed<K> sessionKey, final Headers headers, final StateSerdes<K, ?> serdes) { | ||
| return keyBytes(sessionKey.key(), headers, serdes); | ||
| } | ||
|
|
||
| /** | ||
| * Extract the raw aggregation bytes from serialized AggregationWithHeaders, | ||
| * stripping the headers prefix. | ||
|
|
||
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.
Just some cleanup -- there is no value to have
typedQueryResult.