-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-10546: Deprecate old PAPI #10869
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
2196b2d
d62d82d
41755f1
193b691
443179a
870d207
5e60738
c261a33
e871030
04ab7bd
d9c19db
da30f69
a0d46db
a77cf40
a33ae96
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -487,7 +487,8 @@ public synchronized <K, V> GlobalKTable<K, V> globalTable(final String topic, | |
| /** | ||
| * Adds a state store to the underlying {@link Topology}. | ||
| * <p> | ||
| * It is required to connect state stores to {@link org.apache.kafka.streams.processor.Processor Processors}, {@link Transformer Transformers}, | ||
| * It is required to connect state stores to {@link org.apache.kafka.streams.processor.api.Processor Processors}, | ||
|
Contributor
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. Switching to the new interface where possible. |
||
| * {@link Transformer Transformers}, | ||
| * or {@link ValueTransformer ValueTransformers} before they can be used. | ||
| * | ||
| * @param builder the builder used to obtain this state store {@link StateStore} instance | ||
|
|
@@ -515,7 +516,8 @@ public synchronized StreamsBuilder addStateStore(final StoreBuilder<?> builder) | |
| * This {@link ProcessorNode} should be used to keep the {@link StateStore} up-to-date. | ||
| * The default {@link TimestampExtractor} as specified in the {@link StreamsConfig config} is used. | ||
| * <p> | ||
| * It is not required to connect a global store to {@link org.apache.kafka.streams.processor.Processor Processors}, {@link Transformer Transformers}, | ||
| * It is not required to connect a global store to {@link org.apache.kafka.streams.processor.api.Processor Processors}, | ||
| * {@link Transformer Transformers}, | ||
| * or {@link ValueTransformer ValueTransformer}; those have read-only access to all global stores by default. | ||
| * <p> | ||
| * The supplier should always generate a new instance each time {@link ProcessorSupplier#get()} gets called. Creating | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,11 +22,11 @@ | |
| import org.apache.kafka.streams.kstream.KStream; | ||
| import org.apache.kafka.streams.kstream.KTable; | ||
| import org.apache.kafka.streams.processor.ConnectedStoreProvider; | ||
| import org.apache.kafka.streams.processor.Processor; | ||
| import org.apache.kafka.streams.processor.StateStore; | ||
| import org.apache.kafka.streams.processor.StreamPartitioner; | ||
| import org.apache.kafka.streams.processor.TimestampExtractor; | ||
| import org.apache.kafka.streams.processor.TopicNameExtractor; | ||
| import org.apache.kafka.streams.processor.api.Processor; | ||
| import org.apache.kafka.streams.processor.api.ProcessorSupplier; | ||
| import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; | ||
| import org.apache.kafka.streams.processor.internals.ProcessorAdapter; | ||
|
|
@@ -655,7 +655,7 @@ public synchronized <K, V> Topology addSink(final String name, | |
| * will be added to the topology and connected to this processor automatically. | ||
| * | ||
| * @param name the unique name of the processor node | ||
| * @param supplier the supplier used to obtain this node's {@link Processor} instance | ||
| * @param supplier the supplier used to obtain this node's {@link org.apache.kafka.streams.processor.Processor} instance | ||
|
Contributor
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. Specifying old PAPI by fully qualified name where necessary. |
||
| * @param parentNames the name of one or more source or processor nodes whose output records this processor should receive | ||
| * and process | ||
| * @return itself | ||
|
|
||
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,7 +38,6 @@ | |
| import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters; | ||
| import org.apache.kafka.streams.kstream.internals.graph.StatefulProcessorNode; | ||
| import org.apache.kafka.streams.kstream.internals.graph.GraphNode; | ||
| import org.apache.kafka.streams.processor.ProcessorSupplier; | ||
| import org.apache.kafka.streams.state.StoreBuilder; | ||
|
|
||
| class CogroupedStreamAggregateBuilder<K, VOut> { | ||
|
|
@@ -255,10 +254,11 @@ <KR, VIn> KTable<KR, VOut> createTable(final Collection<GraphNode> processors, | |
| builder); | ||
| } | ||
|
|
||
| @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. | ||
|
Contributor
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. Suppressing the deprecation warnings on our internal processors, and their tests, makes up the lion's share of this PR. I've marked them all with this comment, and filed https://issues.apache.org/jira/browse/KAFKA-12939 to make sure we really migrate everything. |
||
| private StatefulProcessorNode<K, ?> getStatefulProcessorNode(final String processorName, | ||
| final boolean stateCreated, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final ProcessorSupplier<K, ?> kStreamAggregate) { | ||
| final org.apache.kafka.streams.processor.ProcessorSupplier<K, ?> kStreamAggregate) { | ||
| final StatefulProcessorNode<K, ?> statefulProcessorNode; | ||
| if (!stateCreated) { | ||
| statefulProcessorNode = | ||
|
|
||
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.
One super frustrating thing in this PR was that you can't suppress deprecation warnings on imports, so I have had to use the fully qualified class name of the deprecated APIs everywhere.
Aside from a lot of noise in the PR (sorry about that), this means that we have some pretty long lines. It's not always easy to get these lines under the limit without making them even harder to read due to weird line breaks. I've followed my best judgement about when it's better to just keep a line long.