-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-20132: Implement TimestampedKeyValueStoreWithHeaders (3/N) #21451
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
Open
aliehsaeedii
wants to merge
7
commits into
apache:trunk
Choose a base branch
from
aliehsaeedii:metered
base: trunk
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+608
−3
Open
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
574616a
add metered layer
aliehsaeedii 122dd05
metered lyer impl
aliehsaeedii 216c4a1
address reviews
aliehsaeedii 20863ea
Merge branch 'trunk' into metered
aliehsaeedii f0bc89e
address reviews- p1
aliehsaeedii 47bc204
address reviews - p2
aliehsaeedii 19c6be1
Merge branch 'trunk' into metered
aliehsaeedii File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
27 changes: 27 additions & 0 deletions
27
...ams/src/main/java/org/apache/kafka/streams/state/TimestampedKeyValueStoreWithHeaders.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,27 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| /** | ||
| * 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>> { | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
78 changes: 78 additions & 0 deletions
78
.../org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreWithHeaders.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,78 @@ | ||
| /* | ||
| * 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.serialization.Serde; | ||
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.apache.kafka.common.utils.Time; | ||
| import org.apache.kafka.streams.processor.internals.SerdeGetter; | ||
| import org.apache.kafka.streams.query.Position; | ||
| import org.apache.kafka.streams.query.PositionBound; | ||
| import org.apache.kafka.streams.query.Query; | ||
| import org.apache.kafka.streams.query.QueryConfig; | ||
| import org.apache.kafka.streams.query.QueryResult; | ||
| import org.apache.kafka.streams.state.KeyValueStore; | ||
| import org.apache.kafka.streams.state.TimestampedKeyValueStoreWithHeaders; | ||
| import org.apache.kafka.streams.state.ValueTimestampHeaders; | ||
|
|
||
|
|
||
| /** | ||
| * A Metered {@link TimestampedKeyValueStoreWithHeaders} wrapper that is used for recording operation metrics, and hence | ||
| * its inner KeyValueStore implementation does not need to provide its own metrics collecting functionality. | ||
| * | ||
| * The inner {@link KeyValueStore} of this class is of type <Bytes, byte[]>, | ||
| * hence we use {@link Serde}s to convert from <K, ValueTimestampHeaders<V>> to <Bytes, byte[]>. | ||
| * | ||
| * @param <K> key type | ||
| * @param <V> value type (wrapped in {@link ValueTimestampHeaders}) | ||
| */ | ||
| public class MeteredTimestampedKeyValueStoreWithHeaders<K, V> | ||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| extends MeteredKeyValueStore<K, ValueTimestampHeaders<V>> | ||
| implements TimestampedKeyValueStoreWithHeaders<K, V> { | ||
|
|
||
| MeteredTimestampedKeyValueStoreWithHeaders(final KeyValueStore<Bytes, byte[]> inner, | ||
| final String metricScope, | ||
| final Time time, | ||
| final Serde<K> keySerde, | ||
| final Serde<ValueTimestampHeaders<V>> valueSerde) { | ||
| super(inner, metricScope, time, keySerde, valueSerde); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Override | ||
| protected Serde<ValueTimestampHeaders<V>> prepareValueSerdeForStore(final Serde<ValueTimestampHeaders<V>> valueSerde, | ||
| final SerdeGetter getter) { | ||
| if (valueSerde == null) { | ||
| return new ValueTimestampHeadersSerde<>((Serde<V>) getter.valueSerde()); | ||
| } else { | ||
| return super.prepareValueSerdeForStore(valueSerde, getter); | ||
| } | ||
| } | ||
|
|
||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| @Override | ||
| public <R> QueryResult<R> query(final Query<R> query, | ||
| final PositionBound positionBound, | ||
| final QueryConfig config) { | ||
| throw new UnsupportedOperationException("Querying is not supported for " + getClass().getSimpleName()); | ||
| } | ||
|
|
||
| @Override | ||
| public Position getPosition() { | ||
| throw new UnsupportedOperationException("Position is not supported for " + getClass().getSimpleName()); | ||
| } | ||
|
|
||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Looking into the POC PR, we actually added
Headersparameter here, and I think it would be the right thing to do, and just pass innew RecrodHeaders()when used insideMeteredKeyValueStorebut pass in the extractedHeadersobject from the value in theMeteredTimestampedKeyValueWithHeadersStorecase?This implies we need to overload
put()inMeteredTimestampedKeyValueWithHeadersStore.