-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-7658: Add KStream#toTable to the Streams DSL #7985
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
Merged
Merged
Changes from all commits
Commits
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
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
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
67 changes: 67 additions & 0 deletions
67
...ams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.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,67 @@ | ||
| /* | ||
| * 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.kstream.internals.graph; | ||
|
|
||
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.apache.kafka.streams.kstream.internals.KTableSource; | ||
| import org.apache.kafka.streams.kstream.internals.MaterializedInternal; | ||
| import org.apache.kafka.streams.kstream.internals.TimestampedKeyValueStoreMaterializer; | ||
| import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; | ||
| import org.apache.kafka.streams.state.KeyValueStore; | ||
| import org.apache.kafka.streams.state.StoreBuilder; | ||
| import org.apache.kafka.streams.state.TimestampedKeyValueStore; | ||
|
|
||
| /** | ||
| * Represents a KTable convert From KStream | ||
| */ | ||
| public class StreamToTableNode<K, V> extends StreamsGraphNode { | ||
|
|
||
| private final ProcessorParameters<K, V> processorParameters; | ||
| private final MaterializedInternal<K, V, ?> materializedInternal; | ||
|
|
||
| public StreamToTableNode(final String nodeName, | ||
| final ProcessorParameters<K, V> processorParameters, | ||
| final MaterializedInternal<K, V, ?> materializedInternal) { | ||
| super(nodeName); | ||
| this.processorParameters = processorParameters; | ||
| this.materializedInternal = materializedInternal; | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "StreamToTableNode{" + | ||
| ", processorParameters=" + processorParameters + | ||
| ", materializedInternal=" + materializedInternal + | ||
| "} " + super.toString(); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Override | ||
| public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { | ||
| final StoreBuilder<TimestampedKeyValueStore<K, V>> storeBuilder = | ||
| new TimestampedKeyValueStoreMaterializer<>((MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>>) materializedInternal).materialize(); | ||
|
|
||
| final String processorName = processorParameters.processorName(); | ||
| final KTableSource<K, V> ktableSource = (KTableSource<K, V>) processorParameters.processorSupplier(); | ||
| topologyBuilder.addProcessor(processorName, processorParameters.processorSupplier(), parentNodeNames()); | ||
|
|
||
| if (storeBuilder != null && ktableSource.queryableName() != null) { | ||
| topologyBuilder.addStateStore(storeBuilder, processorName); | ||
| } | ||
| } | ||
| } |
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.
Given the change that we might insert a repartition topic in the key was change upstream, we should extend the JavaDocs accordingly. Compare the JavaDocs of
groupByKey()that contain a paragraph about auto-repartitioning. We should add a similar (or maybe even the exact some if suitable) for all four overloads oftoTable(), too.