From f7aaa6393a700b79f612fec8fc061842edf4037a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 19 Mar 2021 13:14:15 +0000 Subject: [PATCH 01/40] add new abstract processor --- .../processor/api/AbstractProcessor.java | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java new file mode 100644 index 0000000000000..f4c03c7e0ec09 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java @@ -0,0 +1,58 @@ +/* + * 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.processor.api; + +/** + * An abstract implementation of {@link Processor} that manages the {@link ProcessorContext} instance and provides default no-op + * implementation of {@link #close()}. + * + * @param the type of input keys + * @param the type of input values + * @param the type of output keys + * @param the type of output values + */ +public abstract class AbstractProcessor implements Processor { + + protected ProcessorContext context; + + protected AbstractProcessor() {} + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + /** + * Close this processor and clean up any resources. + *

+ * This method does nothing by default; if desired, subclasses should override it with custom functionality. + *

+ */ + @Override + public void close() { + // do nothing + } + + /** + * Get the processor's context set during {@link #init(ProcessorContext) initialization}. + * + * @return the processor context; null only when called prior to {@link #init(ProcessorContext) initialization}. + */ + protected final ProcessorContext context() { + return context; + } +} From bed7c96098a6050403006f10936bf28042aa5856 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 23 Mar 2021 11:58:56 +0000 Subject: [PATCH 02/40] migrate kstream flat map values to new processor --- .../kafka/streams/kstream/internals/KStreamFlatMapValues.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 1008b297b3d3c..4475c191083a3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -17,7 +17,11 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapperWithKey; +<<<<<<< HEAD import org.apache.kafka.streams.processor.api.ContextualProcessor; +======= +import org.apache.kafka.streams.processor.api.AbstractProcessor; +>>>>>>> 5d5216288a (migrate kstream flat map values to new processor) import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; From 96576016cf7f69274eb59131cf3da79a6f211d8f Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 30 Mar 2021 00:00:46 +0100 Subject: [PATCH 03/40] apply suggestions --- .../internals/KStreamFlatMapValues.java | 4 -- .../processor/api/AbstractProcessor.java | 58 ------------------- ...va => ContextualProcessorContextTest.java} | 0 3 files changed, 62 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java rename streams/src/test/java/org/apache/kafka/streams/processor/internals/{AbstractProcessorContextTest.java => ContextualProcessorContextTest.java} (100%) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 4475c191083a3..1008b297b3d3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -17,11 +17,7 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -<<<<<<< HEAD import org.apache.kafka.streams.processor.api.ContextualProcessor; -======= -import org.apache.kafka.streams.processor.api.AbstractProcessor; ->>>>>>> 5d5216288a (migrate kstream flat map values to new processor) import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java deleted file mode 100644 index f4c03c7e0ec09..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/processor/api/AbstractProcessor.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.processor.api; - -/** - * An abstract implementation of {@link Processor} that manages the {@link ProcessorContext} instance and provides default no-op - * implementation of {@link #close()}. - * - * @param the type of input keys - * @param the type of input values - * @param the type of output keys - * @param the type of output values - */ -public abstract class AbstractProcessor implements Processor { - - protected ProcessorContext context; - - protected AbstractProcessor() {} - - @Override - public void init(final ProcessorContext context) { - this.context = context; - } - - /** - * Close this processor and clean up any resources. - *

- * This method does nothing by default; if desired, subclasses should override it with custom functionality. - *

- */ - @Override - public void close() { - // do nothing - } - - /** - * Get the processor's context set during {@link #init(ProcessorContext) initialization}. - * - * @return the processor context; null only when called prior to {@link #init(ProcessorContext) initialization}. - */ - protected final ProcessorContext context() { - return context; - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ContextualProcessorContextTest.java similarity index 100% rename from streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java rename to streams/src/test/java/org/apache/kafka/streams/processor/internals/ContextualProcessorContextTest.java From 9983c9cfbcade0de57763204eb3dfff535193d2f Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 31 Mar 2021 14:13:22 +0100 Subject: [PATCH 04/40] rollback change in test class --- ...rocessorContextTest.java => AbstractProcessorContextTest.java} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename streams/src/test/java/org/apache/kafka/streams/processor/internals/{ContextualProcessorContextTest.java => AbstractProcessorContextTest.java} (100%) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ContextualProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java similarity index 100% rename from streams/src/test/java/org/apache/kafka/streams/processor/internals/ContextualProcessorContextTest.java rename to streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java From a619816b024bb82c5590a30ec4bcf76c56b949ce Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 1 Apr 2021 10:03:32 +0100 Subject: [PATCH 05/40] add new processor operator to kstream --- .../apache/kafka/streams/kstream/KStream.java | 47 +++++++++++-------- .../kstream/internals/KStreamImpl.java | 37 +++++++++++++-- .../kstream/internals/KStreamImplTest.java | 9 ++-- 3 files changed, 64 insertions(+), 29 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 00b9ee8991477..e58114a83b37c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -26,8 +26,8 @@ import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.processor.ConnectedStoreProvider; import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.state.KeyValueStore; @@ -3173,12 +3173,12 @@ KStream leftJoin(final GlobalKTable globalTable, * (cf. {@link #transformValues(ValueTransformerSupplier, String...) transformValues()} ) *

* Note that it is possible to emit multiple records for each input record by using - * {@link ProcessorContext#forward(Object, Object) context#forward()} in + * {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} in * {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * If in {@link Transformer#transform(Object, Object) Transformer#transform()} multiple records need to be emitted @@ -3300,12 +3300,12 @@ KStream transform(final TransformerSupplier * Note that it is possible to emit multiple records for each input record by using - * {@link ProcessorContext#forward(Object, Object) context#forward()} in + * {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} in * {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * If in {@link Transformer#transform(Object, Object) Transformer#transform()} multiple records need to be emitted @@ -3432,12 +3432,12 @@ KStream transform(final TransformerSupplier - * Note that it is possible to emit records by using {@link ProcessorContext#forward(Object, Object) + * Note that it is possible to emit records by using {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating @@ -3558,12 +3558,12 @@ KStream flatTransform(final TransformerSupplier - * Note that it is possible to emit records by using {@link ProcessorContext#forward(Object, Object) + * Note that it is possible to emit records by using {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating @@ -3649,7 +3649,7 @@ KStream flatTransform(final TransformerSupplier{@code @@ -3757,7 +3757,7 @@ KStream transformValues(final ValueTransformerSupplier{@code @@ -3870,7 +3870,7 @@ KStream transformValues(final ValueTransformerSupplier{@code @@ -3982,7 +3982,7 @@ KStream transformValues(final ValueTransformerWithKeySupplier{@code @@ -4099,7 +4099,7 @@ KStream transformValues(final ValueTransformerWithKeySupplier{@code @@ -4221,7 +4221,7 @@ KStream flatTransformValues(final ValueTransformerSupplier{@code @@ -4345,7 +4345,7 @@ KStream flatTransformValues(final ValueTransformerSupplier{@code @@ -4468,7 +4468,7 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier{@code @@ -4615,7 +4615,7 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier processorSupplier, + void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, final String... stateStoreNames); /** @@ -4709,7 +4709,14 @@ void process(final ProcessorSupplier processorSupplier, * @see #foreach(ForeachAction) * @see #transform(TransformerSupplier, String...) */ - void process(final ProcessorSupplier processorSupplier, + void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, final Named named, final String... stateStoreNames); + + void process(final ProcessorSupplier processorSupplier, + final String... stateStoreNames); + + void process(final ProcessorSupplier processorSupplier, + final Named named, + final String... stateStoreNames); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index c013bd63a916f..cc468e5dd3113 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -58,7 +58,7 @@ import org.apache.kafka.streams.kstream.internals.graph.UnoptimizableRepartitionNode; import org.apache.kafka.streams.kstream.internals.graph.UnoptimizableRepartitionNode.UnoptimizableRepartitionNodeBuilder; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.kstream.ForeachProcessor; @@ -1213,7 +1213,7 @@ private KStream globalTableJoin(final GlobalKTable g final KTableValueGetterSupplier valueGetterSupplier = ((GlobalKTableImpl) globalTable).valueGetterSupplier(); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, LEFTJOIN_NAME); - final ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( + final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( valueGetterSupplier, joiner, keySelector, @@ -1249,7 +1249,7 @@ private KStream doStreamTableJoin(final KTable table, final NamedInternal renamed = new NamedInternal(joinedInternal.name()); final String name = renamed.orElseGenerateWithPrefix(builder, leftJoin ? LEFTJOIN_NAME : JOIN_NAME); - final ProcessorSupplier processorSupplier = new KStreamKTableJoin<>( + final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier = new KStreamKTableJoin<>( ((KTableImpl) table).valueGetterSupplier(), joiner, leftJoin); @@ -1464,13 +1464,13 @@ private KStream doFlatTransformValues(final ValueTransformerWithKeyS } @Override - public void process(final ProcessorSupplier processorSupplier, + public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, final String... stateStoreNames) { process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); } @Override - public void process(final ProcessorSupplier processorSupplier, + public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, final Named named, final String... stateStoreNames) { Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); @@ -1489,4 +1489,31 @@ public void process(final ProcessorSupplier processorSuppl builder.addGraphNode(graphNode, processNode); } + + @Override + public void process(final ProcessorSupplier processorSupplier, + final String... stateStoreNames) { + process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); + } + + @Override + public void process(final ProcessorSupplier processorSupplier, + final Named named, + final String... stateStoreNames) { + Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); + Objects.requireNonNull(named, "named can't be null"); + Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); + ApiUtils.checkSupplier(processorSupplier); + for (final String stateStoreName : stateStoreNames) { + Objects.requireNonNull(stateStoreName, "stateStoreNames can't be null"); + } + + final String name = new NamedInternal(named).name(); + final StatefulProcessorNode processNode = new StatefulProcessorNode<>( + name, + new ProcessorParameters<>(processorSupplier, name), + stateStoreNames); + + builder.addGraphNode(graphNode, processNode); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index ac5db68b5403b..1ca13ce6d7669 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -55,6 +55,7 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.SourceNode; @@ -2381,7 +2382,7 @@ public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueWithKeySupp public void shouldNotAllowNullProcessSupplierOnProcess() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null)); + () -> testStream.process((ProcessorSupplier) null)); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2389,7 +2390,7 @@ public void shouldNotAllowNullProcessSupplierOnProcess() { public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null, "storeName")); + () -> testStream.process((ProcessorSupplier) null, "storeName")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2397,7 +2398,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null, Named.as("processor"))); + () -> testStream.process((ProcessorSupplier) null, Named.as("processor"))); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2405,7 +2406,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamedAndStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null, Named.as("processor"), "stateStore")); + () -> testStream.process((ProcessorSupplier) null, Named.as("processor"), "stateStore")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } From 19c8a8f0d7a6f5575630cfd85bc75fc3689aa452 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 1 Apr 2021 12:33:15 +0100 Subject: [PATCH 06/40] draft ktable mapvalues, filter and others --- .../AggregationProcessorSupplier.java | 36 +++++++++ .../CogroupedStreamAggregateBuilder.java | 2 +- .../KTableChangeProcessorSupplier.java | 40 ++++++++++ .../kstream/internals/KTableFilter.java | 55 ++++++------- .../streams/kstream/internals/KTableImpl.java | 31 +++++--- .../kstream/internals/KTableMapValues.java | 78 +++++++++---------- ...alizedValueAndTimestampGetterSupplier.java | 52 +++++++++++++ .../kstream/internals/KTablePassThrough.java | 32 ++++---- ...SourceValueAndTimestampGetterSupplier.java | 52 +++++++++++++ .../KTableValueAndTimestampGetter.java | 29 +++++++ ...KTableValueAndTimestampGetterSupplier.java | 24 ++++++ .../TupleChangeCacheFlushListener.java | 55 +++++++++++++ .../internals/TupleChangeForwarder.java | 64 +++++++++++++++ .../internals/AbstractProcessorContext.java | 2 +- .../internals/InternalProcessorContext.java | 4 +- 15 files changed, 461 insertions(+), 95 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueAndTimestampGetterSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueAndTimestampGetterSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetter.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetterSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java new file mode 100644 index 0000000000000..027384569cc5c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java @@ -0,0 +1,36 @@ +/* + * 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; + +import org.apache.kafka.streams.processor.api.ProcessorSupplier; + +/** + * + * @param + * @param + * @param + * @param Table tuple + * @param + */ +public interface AggregationProcessorSupplier extends + ProcessorSupplier> { + + KTableValueAndTimestampGetterSupplier view(); + + void enableSendingOldValues(); +} + diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java index c7585263a718a..1c5115d1dc639 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java @@ -237,7 +237,7 @@ KTable createTable(final Collection processors, "-cogroup-merge", builder, CogroupedKStreamImpl.MERGE_NAME); - final KTableProcessorSupplier passThrough = new KTablePassThrough<>(parentProcessors, storeName); + final KTableChangeProcessorSupplier passThrough = new KTablePassThrough<>(parentProcessors, storeName); final ProcessorParameters processorParameters = new ProcessorParameters(passThrough, mergeProcessorName); final ProcessorGraphNode mergeNode = new ProcessorGraphNode<>(mergeProcessorName, processorParameters); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java new file mode 100644 index 0000000000000..674a692292025 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java @@ -0,0 +1,40 @@ +/* + * 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; + +import org.apache.kafka.streams.processor.api.ProcessorSupplier; + +public interface KTableChangeProcessorSupplier extends ProcessorSupplier, KOut, Change> { + + KTableValueAndTimestampGetterSupplier view(); + + /** + * Potentially enables sending old values. + *

+ * If {@code forceMaterialization} is {@code true}, the method will force the materialization of upstream nodes to + * enable sending old values. + *

+ * If {@code forceMaterialization} is {@code false}, the method will only enable the sending of old values if + * an upstream node is already materialized. + * + * @param forceMaterialization indicates if an upstream node should be forced to materialize to enable sending old + * values. + * @return {@code true} is sending old values is enabled, i.e. either because {@code forceMaterialization} was + * {@code true} or some upstream node is materialized. + */ + boolean enableSendingOldValues(boolean forceMaterialization); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java index 1a9d7731aaa23..f84c4ace453cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java @@ -17,15 +17,16 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableFilter implements KTableProcessorSupplier { +class KTableFilter implements KTableChangeProcessorSupplier { private final KTableImpl parent; private final Predicate predicate; private final boolean filterNot; @@ -45,7 +46,7 @@ class KTableFilter implements KTableProcessorSupplier { } @Override - public Processor> get() { + public Processor, K, Change> get() { return new KTableFilterProcessor(); } @@ -86,38 +87,37 @@ private ValueAndTimestamp computeValue(final K key, final ValueAndTimestamp> { + private class KTableFilterProcessor extends ContextualProcessor, K, Change> { private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); if (queryableName != null) { - store = (TimestampedKeyValueStore) context.getStateStore(queryableName); - tupleForwarder = new TimestampedTupleForwarder<>( + store = context.getStateStore(queryableName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } } @Override - public void process(final K key, final Change change) { - final V newValue = computeValue(key, change.newValue); - final V oldValue = computeOldValue(key, change); + public void process(final Record> record) { + final V newValue = computeValue(record.key(), record.value().newValue); + final V oldValue = computeOldValue(record.key(), record.value()); if (sendOldValues && oldValue == null && newValue == null) { return; // unnecessary to forward here. } if (queryableName != null) { - store.put(key, ValueAndTimestamp.make(newValue, context().timestamp())); - tupleForwarder.maybeForward(key, newValue, oldValue); + store.put(record.key(), ValueAndTimestamp.make(newValue, record.timestamp())); + tupleForwarder.maybeForward(record, newValue, oldValue); } else { - context().forward(key, new Change<>(newValue, oldValue)); + context().forward(record.withValue(new Change<>(newValue, oldValue))); } } @@ -133,16 +133,17 @@ private V computeOldValue(final K key, final Change change) { } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { // if the KTable is materialized, use the materialized store to return getter value; // otherwise rely on the parent getter and apply filter on-the-fly if (queryableName != null) { - return new KTableMaterializedValueGetterSupplier<>(queryableName); + return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); } else { - return new KTableValueGetterSupplier() { - final KTableValueGetterSupplier parentValueGetterSupplier = parent.valueGetterSupplier(); + return new KTableValueAndTimestampGetterSupplier() { + final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = + parent.valueAndTimestampGetterSupplier(); - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KTableFilterValueGetter(parentValueGetterSupplier.get()); } @@ -155,15 +156,15 @@ public String[] storeNames() { } - private class KTableFilterValueGetter implements KTableValueGetter { - private final KTableValueGetter parentGetter; + private class KTableFilterValueGetter implements KTableValueAndTimestampGetter { + private final KTableValueAndTimestampGetter parentGetter; - KTableFilterValueGetter(final KTableValueGetter parentGetter) { + KTableFilterValueGetter(final KTableValueAndTimestampGetter parentGetter) { this.parentGetter = parentGetter; } @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { parentGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 52f7b5f02f2b3..43c93019332ac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -58,7 +58,7 @@ import org.apache.kafka.streams.kstream.internals.suppress.KTableSuppressProcessorSupplier; import org.apache.kafka.streams.kstream.internals.suppress.NamedSuppressed; import org.apache.kafka.streams.kstream.internals.suppress.SuppressedInternal; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopicProperties; import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor; import org.apache.kafka.streams.state.KeyValueStore; @@ -124,18 +124,18 @@ public class KTableImpl extends AbstractStream implements KTable< private static final String TOPIC_SUFFIX = "-topic"; private static final String SINK_NAME = "KTABLE-SINK-"; - private final ProcessorSupplier processorSupplier; + private final ProcessorSupplier processorSupplier; private final String queryableStoreName; private boolean sendOldValues = false; - public KTableImpl(final String name, + public KTableImpl(final String name, final Serde keySerde, final Serde valueSerde, final Set subTopologySourceNodes, final String queryableStoreName, - final ProcessorSupplier processorSupplier, + final ProcessorSupplier, KOut, Change> processorSupplier, final GraphNode graphNode, final InternalStreamsBuilder builder) { super(name, keySerde, valueSerde, subTopologySourceNodes, graphNode, builder); @@ -179,8 +179,8 @@ private KTable doFilter(final Predicate predicate, } final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, FILTER_NAME); - final KTableProcessorSupplier processorSupplier = - new KTableFilter<>(this, predicate, filterNot, queryableStoreName); + final ProcessorSupplier, K, Change> processorSupplier = new KTableFilter<>(this, predicate, filterNot, + queryableStoreName); final ProcessorParameters processorParameters = unsafeCastProcessorParametersToCompletelyDifferentType( new ProcessorParameters<>(processorSupplier, name) @@ -194,7 +194,7 @@ private KTable doFilter(final Predicate predicate, builder.addGraphNode(this.graphNode, tableNode); - return new KTableImpl<>( + return new KTableImpl( name, keySerde, valueSerde, @@ -292,7 +292,7 @@ private KTable doMapValues(final ValueMapperWithKey processorSupplier = new KTableMapValues<>(this, mapper, queryableStoreName); + final ProcessorSupplier, K, Change> processorSupplier = new KTableMapValues<>(this, mapper, queryableStoreName); // leaving in calls to ITB until building topology with graph @@ -310,7 +310,7 @@ private KTable doMapValues(final ValueMapperWithKey( + return new KTableImpl( name, keySerde, valueSerde, @@ -807,6 +807,19 @@ public KGroupedTable groupBy(final KeyValueMapper valueAndTimestampGetterSupplier() { + if (processorSupplier instanceof KTableSource) { + final KTableSource source = (KTableSource) processorSupplier; + // whenever a source ktable is required for getter, it should be materialized + source.materialize(); + return new KTableSourceValueAndTimestampGetterSupplier<>(source.queryableName()); + } else if (processorSupplier instanceof KStreamAggProcessorSupplier) { + return ((AggregationProcessorSupplier) processorSupplier).view(); + } else { + return ((KTableChangeProcessorSupplier) processorSupplier).view(); + } + } + @SuppressWarnings("unchecked") public KTableValueGetterSupplier valueGetterSupplier() { if (processorSupplier instanceof KTableSource) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java index a325920b47de6..5fb6605093cf0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java @@ -17,23 +17,23 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; - -class KTableMapValues implements KTableProcessorSupplier { - private final KTableImpl parent; - private final ValueMapperWithKey mapper; +class KTableMapValues implements KTableChangeProcessorSupplier { + private final KTableImpl parent; + private final ValueMapperWithKey mapper; private final String queryableName; private boolean sendOldValues = false; - KTableMapValues(final KTableImpl parent, - final ValueMapperWithKey mapper, + KTableMapValues(final KTableImpl parent, + final ValueMapperWithKey mapper, final String queryableName) { this.parent = parent; this.mapper = mapper; @@ -41,21 +41,22 @@ class KTableMapValues implements KTableProcessorSupplier { } @Override - public Processor> get() { + public Processor, K, Change> get() { return new KTableMapValuesProcessor(); } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { // if the KTable is materialized, use the materialized store to return getter value; // otherwise rely on the parent getter and apply map-values on-the-fly if (queryableName != null) { - return new KTableMaterializedValueGetterSupplier<>(queryableName); + return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); } else { - return new KTableValueGetterSupplier() { - final KTableValueGetterSupplier parentValueGetterSupplier = parent.valueGetterSupplier(); + return new KTableValueAndTimestampGetterSupplier() { + final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = + parent.valueAndTimestampGetterSupplier(); - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KTableMapValuesValueGetter(parentValueGetterSupplier.get()); } @@ -81,8 +82,8 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { return sendOldValues; } - private V1 computeValue(final K key, final V value) { - V1 newValue = null; + private VOut computeValue(final K key, final VIn value) { + VOut newValue = null; if (value != null) { newValue = mapper.apply(key, value); @@ -91,8 +92,8 @@ private V1 computeValue(final K key, final V value) { return newValue; } - private ValueAndTimestamp computeValueAndTimestamp(final K key, final ValueAndTimestamp valueAndTimestamp) { - V1 newValue = null; + private ValueAndTimestamp computeValueAndTimestamp(final K key, final ValueAndTimestamp valueAndTimestamp) { + VOut newValue = null; long timestamp = 0; if (valueAndTimestamp != null) { @@ -104,38 +105,37 @@ private ValueAndTimestamp computeValueAndTimestamp(final K key, final ValueA } - private class KTableMapValuesProcessor extends AbstractProcessor> { - private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private class KTableMapValuesProcessor extends ContextualProcessor, K, Change> { + private TimestampedKeyValueStore store; + private TupleChangeForwarder tupleForwarder; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); if (queryableName != null) { - store = (TimestampedKeyValueStore) context.getStateStore(queryableName); - tupleForwarder = new TimestampedTupleForwarder<>( + store = context.getStateStore(queryableName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } } @Override - public void process(final K key, final Change change) { - final V1 newValue = computeValue(key, change.newValue); - final V1 oldValue = computeOldValue(key, change); + public void process(final Record> record) { + final VOut newValue = computeValue(record.key(), record.value().newValue); + final VOut oldValue = computeOldValue(record.key(), record.value()); if (queryableName != null) { - store.put(key, ValueAndTimestamp.make(newValue, context().timestamp())); - tupleForwarder.maybeForward(key, newValue, oldValue); + store.put(record.key(), ValueAndTimestamp.make(newValue, record.timestamp())); + tupleForwarder.maybeForward(record, newValue, oldValue); } else { - context().forward(key, new Change<>(newValue, oldValue)); + context().forward(record.withValue(new Change<>(newValue, oldValue))); } } - private V1 computeOldValue(final K key, final Change change) { + private VOut computeOldValue(final K key, final Change change) { if (!sendOldValues) { return null; } @@ -147,20 +147,20 @@ private V1 computeOldValue(final K key, final Change change) { } - private class KTableMapValuesValueGetter implements KTableValueGetter { - private final KTableValueGetter parentGetter; + private class KTableMapValuesValueGetter implements KTableValueAndTimestampGetter { + private final KTableValueAndTimestampGetter parentGetter; - KTableMapValuesValueGetter(final KTableValueGetter parentGetter) { + KTableMapValuesValueGetter(final KTableValueAndTimestampGetter parentGetter) { this.parentGetter = parentGetter; } @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { parentGetter.init(context); } @Override - public ValueAndTimestamp get(final K key) { + public ValueAndTimestamp get(final K key) { return computeValueAndTimestamp(key, parentGetter.get(key)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueAndTimestampGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueAndTimestampGetterSupplier.java new file mode 100644 index 0000000000000..a111759f7ac2d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueAndTimestampGetterSupplier.java @@ -0,0 +1,52 @@ +/* + * 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; + +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class KTableMaterializedValueAndTimestampGetterSupplier implements KTableValueAndTimestampGetterSupplier { + private final String storeName; + + KTableMaterializedValueAndTimestampGetterSupplier(final String storeName) { + this.storeName = storeName; + } + + public KTableValueAndTimestampGetter get() { + return new KTableMaterializedValueAndTimestampGetter(); + } + + @Override + public String[] storeNames() { + return new String[]{storeName}; + } + + private class KTableMaterializedValueAndTimestampGetter implements KTableValueAndTimestampGetter { + private TimestampedKeyValueStore store; + + @Override + public void init(ProcessorContext context) { + store = context.getStateStore(storeName); + } + + @Override + public ValueAndTimestamp get(final K key) { + return store.get(key); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java index 0e6764a461413..2e9a5afa150ab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java @@ -16,16 +16,17 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import java.util.Collection; -public class KTablePassThrough implements KTableProcessorSupplier { - private final Collection parents; +public class KTablePassThrough implements KTableChangeProcessorSupplier { + private final Collection parents; //TODO change to aggregationprocessor private final String storeName; @@ -35,7 +36,7 @@ public class KTablePassThrough implements KTableProcessorSupplier } @Override - public Processor> get() { + public Processor, K, Change> get() { return new KTablePassThroughProcessor(); } @@ -49,11 +50,11 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { - return new KTableValueGetterSupplier() { + return new KTableValueAndTimestampGetterSupplier() { - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KTablePassThroughValueGetter(); } @@ -64,20 +65,19 @@ public String[] storeNames() { }; } - private class KTablePassThroughProcessor extends AbstractProcessor> { + private class KTablePassThroughProcessor extends ContextualProcessor, K, Change> { @Override - public void process(final K key, final Change value) { - context().forward(key, value); + public void process(final Record> record) { + context().forward(record); } } - private class KTablePassThroughValueGetter implements KTableValueGetter { + private class KTablePassThroughValueGetter implements KTableValueAndTimestampGetter { private TimestampedKeyValueStore store; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - store = (TimestampedKeyValueStore) context.getStateStore(storeName); + public void init(final ProcessorContext context) { + store = context.getStateStore(storeName); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueAndTimestampGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueAndTimestampGetterSupplier.java new file mode 100644 index 0000000000000..4eb10a8207869 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueAndTimestampGetterSupplier.java @@ -0,0 +1,52 @@ +/* + * 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; + +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class KTableSourceValueAndTimestampGetterSupplier implements KTableValueAndTimestampGetterSupplier { + private final String storeName; + + public KTableSourceValueAndTimestampGetterSupplier(final String storeName) { + this.storeName = storeName; + } + + public KTableValueAndTimestampGetter get() { + return new KTableSourceValueGetter(); + } + + @Override + public String[] storeNames() { + return new String[]{storeName}; + } + + private class KTableSourceValueGetter implements KTableValueAndTimestampGetter { + private TimestampedKeyValueStore store = null; + + @Override + public void init(ProcessorContext context) { + store = context.getStateStore(storeName); + } + + public ValueAndTimestamp get(final K key) { + return store.get(key); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetter.java new file mode 100644 index 0000000000000..b3ba7de5f8cd3 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetter.java @@ -0,0 +1,29 @@ +/* + * 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; + +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public interface KTableValueAndTimestampGetter { + + void init(ProcessorContext context); + + ValueAndTimestamp get(K key); + + default void close() {} +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetterSupplier.java new file mode 100644 index 0000000000000..89a8b998b19d1 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetterSupplier.java @@ -0,0 +1,24 @@ +/* + * 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; + +public interface KTableValueAndTimestampGetterSupplier { + + KTableValueAndTimestampGetter get(); + + String[] storeNames(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java new file mode 100644 index 0000000000000..241efd3158fff --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java @@ -0,0 +1,55 @@ +/* + * 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; + +import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; + +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorNode; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.CacheFlushListener; + +class TupleChangeCacheFlushListener implements CacheFlushListener> { + + private final InternalProcessorContext> context; + private final ProcessorNode myNode; + + TupleChangeCacheFlushListener(final ProcessorContext> context) { + this.context = (InternalProcessorContext>) context; + myNode = this.context.currentNode(); + } + + @Override + public void apply(final K key, + final ValueAndTimestamp newValue, + final ValueAndTimestamp oldValue, + final long timestamp) { + final ProcessorNode prev = context.currentNode(); + context.setCurrentNode(myNode); + try { + Record> record = new Record<>( + key, + new Change<>(getValueOrNull(newValue), getValueOrNull(oldValue)), + newValue != null ? newValue.timestamp() : timestamp); + context.forward(record); + } finally { + context.setCurrentNode(prev); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java new file mode 100644 index 0000000000000..b2f149318fc3a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java @@ -0,0 +1,64 @@ +/* + * 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; + +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.state.internals.WrappedStateStore; + +/** + * This class is used to determine if a processor should forward values to child nodes. + * Forwarding by this class only occurs when caching is not enabled. If caching is enabled, + * forwarding occurs in the flush listener when the cached store flushes. + * + * @param the type of the key + * @param the type of the value + */ +class TupleChangeForwarder { + private final ProcessorContext> context; + private final boolean sendOldValues; + private final boolean cachingEnabled; + + TupleChangeForwarder(final StateStore store, + final ProcessorContext> context, + final TupleChangeCacheFlushListener flushListener, + final boolean sendOldValues) { + this.context = context; + this.sendOldValues = sendOldValues; + cachingEnabled = ((WrappedStateStore) store).setFlushListener(flushListener, sendOldValues); + } + + public void maybeForward(final Record record, + final V newValue, + final V oldValue) { + if (!cachingEnabled) { + context.forward(record.withValue(new Change<>(newValue, sendOldValues ? oldValue : null))); + } + } + + public void maybeForward(final Record record, + final V newValue, + final V oldValue, + final long timestamp) { + if (!cachingEnabled) { + context.forward(record + .withValue(new Change<>(newValue, sendOldValues ? oldValue : null)) + .withTimestamp(timestamp)); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java index 09a2e31f04dc9..37ffbdc4e6cbd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java @@ -34,7 +34,7 @@ import java.util.Objects; import java.util.Optional; -public abstract class AbstractProcessorContext implements InternalProcessorContext { +public abstract class AbstractProcessorContext implements InternalProcessorContext { private final TaskId taskId; private final String applicationId; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java index ba5c580d78de9..3ffd9469572a6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java @@ -34,8 +34,8 @@ * {@link ProcessorNode} when we are forwarding items that have been evicted or flushed from * {@link ThreadCache} */ -public interface InternalProcessorContext - extends ProcessorContext, org.apache.kafka.streams.processor.api.ProcessorContext, StateStoreContext { +public interface InternalProcessorContext + extends ProcessorContext, org.apache.kafka.streams.processor.api.ProcessorContext, StateStoreContext { BytesSerializer BYTES_KEY_SERIALIZER = new BytesSerializer(); ByteArraySerializer BYTEARRAY_VALUE_SERIALIZER = new ByteArraySerializer(); From 990ba666384bcc8f6e362faf3475b6f7611724db Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 1 Apr 2021 12:48:56 +0100 Subject: [PATCH 07/40] ktablesource to new processor api --- .../kstream/internals/KStreamImpl.java | 2 +- .../streams/kstream/internals/KTableImpl.java | 2 +- .../kstream/internals/KTableSource.java | 53 ++++++++++--------- .../streams/state/ValueAndTimestamp.java | 5 ++ 4 files changed, 35 insertions(+), 27 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index cc468e5dd3113..547aa55f2bccc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -748,7 +748,7 @@ public KTable toTable(final Named named, subTopologySourceNodes = this.subTopologySourceNodes; } - final KTableSource tableSource = new KTableSource<>( + ProcessorSupplier> tableSource = new KTableSource<>( materializedInternal.storeName(), materializedInternal.queryableStoreName() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 43c93019332ac..15150627e58db 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -135,7 +135,7 @@ public KTableImpl(final String name, final Serde valueSerde, final Set subTopologySourceNodes, final String queryableStoreName, - final ProcessorSupplier, KOut, Change> processorSupplier, + final ProcessorSupplier processorSupplier, final GraphNode graphNode, final InternalStreamsBuilder builder) { super(name, keySerde, valueSerde, subTopologySourceNodes, graphNode, builder); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index b9f3580234a15..56cc6fca95977 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -17,10 +17,11 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -31,7 +32,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; -public class KTableSource implements ProcessorSupplier { +public class KTableSource implements ProcessorSupplier> { private static final Logger LOG = LoggerFactory.getLogger(KTableSource.class); private final String storeName; @@ -51,7 +52,7 @@ public String queryableName() { } @Override - public Processor get() { + public Processor> get() { return new KTableSourceProcessor(); } @@ -72,57 +73,59 @@ public boolean materialized() { return queryableName != null; } - private class KTableSourceProcessor extends AbstractProcessor { + private class KTableSourceProcessor extends ContextualProcessor> { private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); if (queryableName != null) { - store = (TimestampedKeyValueStore) context.getStateStore(queryableName); - tupleForwarder = new TimestampedTupleForwarder<>( + store = context.getStateStore(queryableName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } } @Override - public void process(final K key, final V value) { + public void process(final Record record) { // if the key is null, then ignore the record - if (key == null) { - LOG.warn( - "Skipping record due to null key. topic=[{}] partition=[{}] offset=[{}]", - context().topic(), context().partition(), context().offset() - ); + if (record.key() == null) { + // TODO check if context needed for logging +// LOG.warn( +// "Skipping record due to null key. topic=[{}] partition=[{}] offset=[{}]", +// context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } if (queryableName != null) { - final ValueAndTimestamp oldValueAndTimestamp = store.get(key); + final ValueAndTimestamp oldValueAndTimestamp = store.get(record.key()); final V oldValue; if (oldValueAndTimestamp != null) { oldValue = oldValueAndTimestamp.value(); - if (context().timestamp() < oldValueAndTimestamp.timestamp()) { + if (record.timestamp() < oldValueAndTimestamp.timestamp()) { LOG.warn("Detected out-of-order KTable update for {} at offset {}, partition {}.", - store.name(), context().offset(), context().partition()); + store.name(), + context().recordMetadata().get().offset(), + context().recordMetadata().get().partition()); } } else { oldValue = null; } - store.put(key, ValueAndTimestamp.make(value, context().timestamp())); - tupleForwarder.maybeForward(key, value, oldValue); + store.put(record.key(), ValueAndTimestamp.make(record)); + tupleForwarder.maybeForward(record, record.value(), oldValue); } else { - context().forward(key, new Change<>(value, null)); + context().forward(record.withValue(new Change<>(record.value(), null))); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java index f5fc7a2827a41..5ce42e8a0cc20 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java @@ -19,6 +19,7 @@ import org.apache.kafka.streams.KeyValue; import java.util.Objects; +import org.apache.kafka.streams.processor.api.Record; /** * Combines a value from a {@link KeyValue} with a timestamp. @@ -62,6 +63,10 @@ public static V getValueOrNull(final ValueAndTimestamp valueAndTimestamp) return valueAndTimestamp == null ? null : valueAndTimestamp.value(); } + public static ValueAndTimestamp make(Record record) { + return record.value() == null ? null : new ValueAndTimestamp<>(record.value(), record.timestamp()); + } + public V value() { return value; } From fdaf4082f019999d4dd491c1397760c8fd870c37 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 8 Apr 2021 20:06:11 +0100 Subject: [PATCH 08/40] migrate ktable supress, reduce, and aggregate --- .../kstream/internals/KGroupedTableImpl.java | 10 +-- .../kstream/internals/KTableAggregate.java | 53 +++++++-------- .../streams/kstream/internals/KTableImpl.java | 4 +- .../internals/KTableKTableJoinMerger.java | 58 ++++++++-------- .../kstream/internals/KTableReduce.java | 54 +++++++-------- .../internals/KTableTransformValues.java | 59 ++++++++-------- .../internals/graph/KTableKTableJoinNode.java | 5 +- .../internals/graph/ProcessorParameters.java | 7 +- .../KTableSuppressProcessorSupplier.java | 67 ++++++++++--------- 9 files changed, 160 insertions(+), 157 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java index 4dde9c126c033..b418944f76664 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java @@ -29,7 +29,7 @@ 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.processor.api.ProcessorSupplier; import org.apache.kafka.streams.state.KeyValueStore; import java.util.Collections; @@ -68,7 +68,7 @@ public class KGroupedTableImpl extends AbstractStream implements KGr this.userProvidedRepartitionTopicName = groupedInternal.name(); } - private KTable doAggregate(final ProcessorSupplier> aggregateSupplier, + private KTable doAggregate(final ProcessorSupplier, K, Change> aggregateSupplier, final NamedInternal named, final String functionName, final MaterializedInternal> materialized) { @@ -145,7 +145,7 @@ public KTable reduce(final Reducer adder, if (materializedInternal.valueSerde() == null) { materializedInternal.withValueSerde(valueSerde); } - final ProcessorSupplier> aggregateSupplier = new KTableReduce<>( + final ProcessorSupplier, K, Change> aggregateSupplier = new KTableReduce<>( materializedInternal.storeName(), adder, subtractor); @@ -176,7 +176,7 @@ public KTable count(final Named named, final Materialized> aggregateSupplier = new KTableAggregate<>( + final ProcessorSupplier, K, Change> aggregateSupplier = new KTableAggregate<>( materializedInternal.storeName(), countInitializer, countAdder, @@ -221,7 +221,7 @@ public KTable aggregate(final Initializer initializer, if (materializedInternal.keySerde() == null) { materializedInternal.withKeySerde(keySerde); } - final ProcessorSupplier> aggregateSupplier = new KTableAggregate<>( + final ProcessorSupplier, K, Change> aggregateSupplier = new KTableAggregate<>( materializedInternal.storeName(), initializer, adder, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index 1e6474627d6ff..ce409a3acc203 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -19,15 +19,15 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KTableAggregate implements KTableProcessorSupplier { +public class KTableAggregate implements KTableChangeProcessorSupplier { private final String storeName; private final Initializer initializer; @@ -54,23 +54,21 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { } @Override - public Processor> get() { + public Processor, K, Change> get() { return new KTableAggregateProcessor(); } - private class KTableAggregateProcessor extends AbstractProcessor> { + private class KTableAggregateProcessor implements Processor, K, Change> { private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - super.init(context); - store = (TimestampedKeyValueStore) context.getStateStore(storeName); - tupleForwarder = new TimestampedTupleForwarder<>( + public void init(final ProcessorContext> context) { + store = context.getStateStore(storeName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } @@ -78,28 +76,28 @@ public void init(final ProcessorContext context) { * @throws StreamsException if key is null */ @Override - public void process(final K key, final Change value) { + public void process(Record> record) { // the keys should never be null - if (key == null) { + if (record.key() == null) { throw new StreamsException("Record key for KTable aggregate operator with state " + storeName + " should not be null."); } - final ValueAndTimestamp oldAggAndTimestamp = store.get(key); + final ValueAndTimestamp oldAggAndTimestamp = store.get(record.key()); final T oldAgg = getValueOrNull(oldAggAndTimestamp); final T intermediateAgg; - long newTimestamp = context().timestamp(); + long newTimestamp = record.timestamp(); // first try to remove the old value - if (value.oldValue != null && oldAgg != null) { - intermediateAgg = remove.apply(key, value.oldValue, oldAgg); - newTimestamp = Math.max(context().timestamp(), oldAggAndTimestamp.timestamp()); + if (record.value().oldValue != null && oldAgg != null) { + intermediateAgg = remove.apply(record.key(), record.value().oldValue, oldAgg); + newTimestamp = Math.max(record.timestamp(), oldAggAndTimestamp.timestamp()); } else { intermediateAgg = oldAgg; } // then try to add the new value final T newAgg; - if (value.newValue != null) { + if (record.value().newValue != null) { final T initializedAgg; if (intermediateAgg == null) { initializedAgg = initializer.apply(); @@ -107,23 +105,22 @@ public void process(final K key, final Change value) { initializedAgg = intermediateAgg; } - newAgg = add.apply(key, value.newValue, initializedAgg); + newAgg = add.apply(record.key(), record.value().newValue, initializedAgg); if (oldAggAndTimestamp != null) { - newTimestamp = Math.max(context().timestamp(), oldAggAndTimestamp.timestamp()); + newTimestamp = Math.max(record.timestamp(), oldAggAndTimestamp.timestamp()); } } else { newAgg = intermediateAgg; } // update the store with the new value - store.put(key, ValueAndTimestamp.make(newAgg, newTimestamp)); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, newTimestamp); + store.put(record.key(), ValueAndTimestamp.make(newAgg, newTimestamp)); + tupleForwarder.maybeForward(record, newAgg, sendOldValues ? oldAgg : null, newTimestamp); } - } @Override - public KTableValueGetterSupplier view() { - return new KTableMaterializedValueGetterSupplier<>(storeName); + public KTableValueAndTimestampGetterSupplier view() { + return new KTableMaterializedValueAndTimestampGetterSupplier<>(storeName); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 15150627e58db..0f0e086355726 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -445,7 +445,7 @@ private KTable doTransformValues(final ValueTransformerWithKeySuppli final String name = namedInternal.orElseGenerateWithPrefix(builder, TRANSFORMVALUES_NAME); - final KTableProcessorSupplier processorSupplier = new KTableTransformValues<>( + final KTableChangeProcessorSupplier processorSupplier = new KTableTransformValues<>( this, transformerSupplier, queryableStoreName); @@ -526,7 +526,7 @@ public KTable suppress(final Suppressed suppressed) { final String storeName = suppressedInternal.name() != null ? suppressedInternal.name() + "-store" : builder.newStoreName(SUPPRESS_NAME); - final ProcessorSupplier> suppressionSupplier = new KTableSuppressProcessorSupplier<>( + final ProcessorSupplier, K, Change> suppressionSupplier = new KTableSuppressProcessorSupplier<>( suppressedInternal, storeName, this diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java index 3ca0160d24220..e6244aa4ec785 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java @@ -16,9 +16,10 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -26,15 +27,15 @@ import java.util.HashSet; import java.util.Set; -public class KTableKTableJoinMerger implements KTableProcessorSupplier { +public class KTableKTableJoinMerger implements KTableChangeProcessorSupplier { - private final KTableProcessorSupplier parent1; - private final KTableProcessorSupplier parent2; + private final KTableChangeProcessorSupplier parent1; + private final KTableChangeProcessorSupplier parent2; private final String queryableName; private boolean sendOldValues = false; - KTableKTableJoinMerger(final KTableProcessorSupplier parent1, - final KTableProcessorSupplier parent2, + KTableKTableJoinMerger(final KTableChangeProcessorSupplier parent1, + final KTableChangeProcessorSupplier parent2, final String queryableName) { this.parent1 = parent1; this.parent2 = parent2; @@ -46,20 +47,20 @@ public String getQueryableName() { } @Override - public Processor> get() { + public Processor, K, Change> get() { return new KTableKTableJoinMergeProcessor(); } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { // if the result KTable is materialized, use the materialized store to return getter value; // otherwise rely on the parent getter and apply join on-the-fly if (queryableName != null) { - return new KTableMaterializedValueGetterSupplier<>(queryableName); + return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); } else { - return new KTableValueGetterSupplier() { + return new KTableValueAndTimestampGetterSupplier() { - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return parent1.view().get(); } @@ -85,45 +86,44 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { return true; } - public static KTableKTableJoinMerger of(final KTableProcessorSupplier parent1, - final KTableProcessorSupplier parent2) { + public static KTableKTableJoinMerger of(final KTableChangeProcessorSupplier parent1, + final KTableChangeProcessorSupplier parent2) { return of(parent1, parent2, null); } - public static KTableKTableJoinMerger of(final KTableProcessorSupplier parent1, - final KTableProcessorSupplier parent2, + public static KTableKTableJoinMerger of(final KTableChangeProcessorSupplier parent1, + final KTableChangeProcessorSupplier parent2, final String queryableName) { return new KTableKTableJoinMerger<>(parent1, parent2, queryableName); } - private class KTableKTableJoinMergeProcessor extends AbstractProcessor> { + private class KTableKTableJoinMergeProcessor extends ContextualProcessor, K, Change> { private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); if (queryableName != null) { - store = (TimestampedKeyValueStore) context.getStateStore(queryableName); - tupleForwarder = new TimestampedTupleForwarder<>( + store = context.getStateStore(queryableName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } } @Override - public void process(final K key, final Change value) { + public void process(Record> record) { if (queryableName != null) { - store.put(key, ValueAndTimestamp.make(value.newValue, context().timestamp())); - tupleForwarder.maybeForward(key, value.newValue, sendOldValues ? value.oldValue : null); + store.put(record.key(), ValueAndTimestamp.make(record.value().newValue, record.timestamp())); + tupleForwarder.maybeForward(record, record.value().newValue, sendOldValues ? record.value().oldValue : null); } else { if (sendOldValues) { - context().forward(key, value); + context().forward(record); } else { - context().forward(key, new Change<>(value.newValue, null)); + context().forward(record.withValue(new Change<>(record.value().newValue, null))); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index 88f62f96d5d6b..6bc386a1873ff 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -18,15 +18,15 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KTableReduce implements KTableProcessorSupplier { +public class KTableReduce implements KTableChangeProcessorSupplier { private final String storeName; private final Reducer addReducer; @@ -48,24 +48,22 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { } @Override - public Processor> get() { + public Processor, K, Change> get() { return new KTableReduceProcessor(); } - private class KTableReduceProcessor extends AbstractProcessor> { + private class KTableReduceProcessor implements Processor, K, Change> { private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - super.init(context); - store = (TimestampedKeyValueStore) context.getStateStore(storeName); - tupleForwarder = new TimestampedTupleForwarder<>( + public void init(final ProcessorContext> context) { + store = context.getStateStore(storeName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } @@ -73,47 +71,47 @@ public void init(final ProcessorContext context) { * @throws StreamsException if key is null */ @Override - public void process(final K key, final Change value) { + public void process(Record> record) { // the keys should never be null - if (key == null) { + if (record.key() == null) { throw new StreamsException("Record key for KTable reduce operator with state " + storeName + " should not be null."); } - final ValueAndTimestamp oldAggAndTimestamp = store.get(key); + final ValueAndTimestamp oldAggAndTimestamp = store.get(record.key()); final V oldAgg = getValueOrNull(oldAggAndTimestamp); final V intermediateAgg; long newTimestamp; // first try to remove the old value - if (value.oldValue != null && oldAgg != null) { - intermediateAgg = removeReducer.apply(oldAgg, value.oldValue); - newTimestamp = Math.max(context().timestamp(), oldAggAndTimestamp.timestamp()); + if (record.value().oldValue != null && oldAgg != null) { + intermediateAgg = removeReducer.apply(oldAgg, record.value().oldValue); + newTimestamp = Math.max(record.timestamp(), oldAggAndTimestamp.timestamp()); } else { intermediateAgg = oldAgg; - newTimestamp = context().timestamp(); + newTimestamp = record.timestamp(); } // then try to add the new value final V newAgg; - if (value.newValue != null) { + if (record.value().newValue != null) { if (intermediateAgg == null) { - newAgg = value.newValue; + newAgg = record.value().newValue; } else { - newAgg = addReducer.apply(intermediateAgg, value.newValue); - newTimestamp = Math.max(context().timestamp(), oldAggAndTimestamp.timestamp()); + newAgg = addReducer.apply(intermediateAgg, record.value().newValue); + newTimestamp = Math.max(record.timestamp(), oldAggAndTimestamp.timestamp()); } } else { newAgg = intermediateAgg; } // update the store with the new value - store.put(key, ValueAndTimestamp.make(newAgg, newTimestamp)); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, newTimestamp); + store.put(record.key(), ValueAndTimestamp.make(newAgg, newTimestamp)); + tupleForwarder.maybeForward(record, newAgg, sendOldValues ? oldAgg : null, newTimestamp); } } @Override - public KTableValueGetterSupplier view() { - return new KTableMaterializedValueGetterSupplier<>(storeName); + public KTableValueAndTimestampGetterSupplier view() { + return new KTableMaterializedValueAndTimestampGetterSupplier<>(storeName); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java index e4c76d3d4a083..9bbdf6b0ad975 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java @@ -18,10 +18,10 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -30,7 +30,7 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableTransformValues implements KTableProcessorSupplier { +class KTableTransformValues implements KTableChangeProcessorSupplier { private final KTableImpl parent; private final ValueTransformerWithKeySupplier transformerSupplier; private final String queryableName; @@ -45,20 +45,20 @@ class KTableTransformValues implements KTableProcessorSupplier> get() { + public Processor, K, Change> get() { return new KTableTransformValuesProcessor(transformerSupplier.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { if (queryableName != null) { - return new KTableMaterializedValueGetterSupplier<>(queryableName); + return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); } - return new KTableValueGetterSupplier() { - final KTableValueGetterSupplier parentValueGetterSupplier = parent.valueGetterSupplier(); + return new KTableValueAndTimestampGetterSupplier() { + final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = parent.valueAndTimestampGetterSupplier(); - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KTableTransformValuesGetter( parentValueGetterSupplier.get(), transformerSupplier.get()); @@ -84,10 +84,11 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { return sendOldValues; } - private class KTableTransformValuesProcessor extends AbstractProcessor> { + private class KTableTransformValuesProcessor extends + ContextualProcessor, K, Change> { private final ValueTransformerWithKey valueTransformer; private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; private KTableTransformValuesProcessor(final ValueTransformerWithKey valueTransformer) { this.valueTransformer = Objects.requireNonNull(valueTransformer, "valueTransformer"); @@ -95,30 +96,30 @@ private KTableTransformValuesProcessor(final ValueTransformerWithKey> context) { super.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext(context)); +// valueTransformer.init(new ForwardingDisabledProcessorContext(context)); if (queryableName != null) { store = (TimestampedKeyValueStore) context.getStateStore(queryableName); - tupleForwarder = new TimestampedTupleForwarder<>( + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } } @Override - public void process(final K key, final Change change) { - final V1 newValue = valueTransformer.transform(key, change.newValue); + public void process(Record> record) { + final V1 newValue = valueTransformer.transform(record.key(), record.value().newValue); if (queryableName == null) { - final V1 oldValue = sendOldValues ? valueTransformer.transform(key, change.oldValue) : null; - context().forward(key, new Change<>(newValue, oldValue)); + final V1 oldValue = sendOldValues ? valueTransformer.transform(record.key(), record.value().oldValue) : null; + context().forward(record.withValue(new Change<>(newValue, oldValue))); } else { - final V1 oldValue = sendOldValues ? getValueOrNull(store.get(key)) : null; - store.put(key, ValueAndTimestamp.make(newValue, context().timestamp())); - tupleForwarder.maybeForward(key, newValue, oldValue); + final V1 oldValue = sendOldValues ? getValueOrNull(store.get(record.key())) : null; + store.put(record.key(), ValueAndTimestamp.make(newValue, record.timestamp())); + tupleForwarder.maybeForward(record, newValue, oldValue); } } @@ -129,20 +130,20 @@ public void close() { } - private class KTableTransformValuesGetter implements KTableValueGetter { - private final KTableValueGetter parentGetter; + private class KTableTransformValuesGetter implements KTableValueAndTimestampGetter { + private final KTableValueAndTimestampGetter parentGetter; private final ValueTransformerWithKey valueTransformer; - KTableTransformValuesGetter(final KTableValueGetter parentGetter, + KTableTransformValuesGetter(final KTableValueAndTimestampGetter parentGetter, final ValueTransformerWithKey valueTransformer) { this.parentGetter = Objects.requireNonNull(parentGetter, "parentGetter"); this.valueTransformer = Objects.requireNonNull(valueTransformer, "valueTransformer"); } @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { parentGetter.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext(context)); +// valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java index e2abfb5d38a4a..4f43ee2a3ac2c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java @@ -208,7 +208,6 @@ public KTableKTableJoinNodeBuilder withStoreBuilder(final StoreBu return this; } - @SuppressWarnings("unchecked") public KTableKTableJoinNode build() { return new KTableKTableJoinNode<>( nodeName, @@ -216,8 +215,8 @@ public KTableKTableJoinNode build() { joinOtherProcessorParameters, new ProcessorParameters<>( KTableKTableJoinMerger.of( - joinThisProcessorParameters.kTableProcessorSupplier(), - joinOtherProcessorParameters.kTableProcessorSupplier(), + joinThisProcessorParameters.kTableChangeProcessorSupplier(), + joinOtherProcessorParameters.kTableChangeProcessorSupplier(), queryableStoreName), nodeName), thisJoinSide, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java index 018d2b7dc7dd3..51c40d9b3990a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals.graph; +import org.apache.kafka.streams.kstream.internals.KTableChangeProcessorSupplier; import org.apache.kafka.streams.kstream.internals.KTableKTableJoinMerger; import org.apache.kafka.streams.kstream.internals.KTableProcessorSupplier; import org.apache.kafka.streams.kstream.internals.KTableSource; @@ -70,12 +71,16 @@ KTableSource kTableSourceSupplier() { : (KTableSource) oldProcessorSupplier; } - @SuppressWarnings("unchecked") KTableProcessorSupplier kTableProcessorSupplier() { // This cast always works because KTableProcessorSupplier hasn't been converted yet. return (KTableProcessorSupplier) oldProcessorSupplier; } + KTableChangeProcessorSupplier kTableChangeProcessorSupplier() { + // This cast always works because KTableProcessorSupplier hasn't been converted yet. + return (KTableChangeProcessorSupplier) processorSupplier; + } + @SuppressWarnings("unchecked") KTableKTableJoinMerger kTableKTableJoinMergerProcessorSupplier() { return (KTableKTableJoinMerger) oldProcessorSupplier; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java index 6a300b879ece2..44fc5f01ca269 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java @@ -16,19 +16,22 @@ */ package org.apache.kafka.streams.kstream.internals.suppress; +import static java.util.Objects.requireNonNull; + import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.kstream.internals.KTableChangeProcessorSupplier; import org.apache.kafka.streams.kstream.internals.KTableImpl; -import org.apache.kafka.streams.kstream.internals.KTableProcessorSupplier; -import org.apache.kafka.streams.kstream.internals.KTableValueGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; import org.apache.kafka.streams.kstream.internals.suppress.TimeDefinitions.TimeDefinition; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics; @@ -36,16 +39,16 @@ import org.apache.kafka.streams.state.internals.Maybe; import org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBuffer; -import static java.util.Objects.requireNonNull; +public class KTableSuppressProcessorSupplier implements + KTableChangeProcessorSupplier { -public class KTableSuppressProcessorSupplier implements KTableProcessorSupplier { private final SuppressedInternal suppress; private final String storeName; private final KTableImpl parentKTable; public KTableSuppressProcessorSupplier(final SuppressedInternal suppress, - final String storeName, - final KTableImpl parentKTable) { + final String storeName, + final KTableImpl parentKTable) { this.suppress = suppress; this.storeName = storeName; this.parentKTable = parentKTable; @@ -54,27 +57,26 @@ public KTableSuppressProcessorSupplier(final SuppressedInternal suppress, } @Override - public Processor> get() { + public Processor, K, Change> get() { return new KTableSuppressProcessor<>(suppress, storeName); } @Override - public KTableValueGetterSupplier view() { - final KTableValueGetterSupplier parentValueGetterSupplier = parentKTable.valueGetterSupplier(); - return new KTableValueGetterSupplier() { + public KTableValueAndTimestampGetterSupplier view() { + final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = parentKTable.valueAndTimestampGetterSupplier(); + return new KTableValueAndTimestampGetterSupplier() { @Override - public KTableValueGetter get() { - final KTableValueGetter parentGetter = parentValueGetterSupplier.get(); - return new KTableValueGetter() { + public KTableValueAndTimestampGetter get() { + final KTableValueAndTimestampGetter parentGetter = parentValueGetterSupplier.get(); + return new KTableValueAndTimestampGetter() { private TimeOrderedKeyValueBuffer buffer; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { parentGetter.init(context); // the main processor is responsible for the buffer's lifecycle - buffer = requireNonNull((TimeOrderedKeyValueBuffer) context.getStateStore(storeName)); + buffer = requireNonNull(context.getStateStore(storeName)); } @Override @@ -112,7 +114,8 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { return parentKTable.enableSendingOldValues(forceMaterialization); } - private static final class KTableSuppressProcessor extends AbstractProcessor> { + private static final class KTableSuppressProcessor extends + ContextualProcessor, K, Change> { private final long maxRecords; private final long maxBytes; private final long suppressDurationMillis; @@ -122,7 +125,7 @@ private static final class KTableSuppressProcessor extends AbstractProcess private final String storeName; private TimeOrderedKeyValueBuffer buffer; - private InternalProcessorContext internalProcessorContext; + private InternalProcessorContext> internalProcessorContext; private Sensor suppressionEmitSensor; private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; @@ -139,9 +142,9 @@ private KTableSuppressProcessor(final SuppressedInternal suppress, final Stri @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.api.ProcessorContext> context) { super.init(context); - internalProcessorContext = (InternalProcessorContext) context; + internalProcessorContext = (InternalProcessorContext>) context; suppressionEmitSensor = ProcessorNodeMetrics.suppressionEmitSensor( Thread.currentThread().getName(), context.taskId().toString(), @@ -149,17 +152,10 @@ public void init(final ProcessorContext context) { internalProcessorContext.metrics() ); - buffer = requireNonNull((TimeOrderedKeyValueBuffer) context.getStateStore(storeName)); + buffer = requireNonNull(context.getStateStore(storeName)); buffer.setSerdesIfNull((Serde) context.keySerde(), (Serde) context.valueSerde()); } - @Override - public void process(final K key, final Change value) { - observedStreamTime = Math.max(observedStreamTime, internalProcessorContext.timestamp()); - buffer(key, value); - enforceConstraints(); - } - private void buffer(final K key, final Change value) { final long bufferTime = bufferTimeDefinition.time(internalProcessorContext, key); @@ -213,5 +209,12 @@ private void emit(final TimeOrderedKeyValueBuffer.Eviction toEmit) { private boolean shouldForward(final Change value) { return value.newValue != null || !safeToDropTombstones; } + + @Override + public void process(Record> record) { + observedStreamTime = Math.max(observedStreamTime, internalProcessorContext.timestamp()); + buffer(record.key(), record.value()); + enforceConstraints(); + } } } From ae995d2e77996665f870bfbe93ce0672dbe1935e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 8 Apr 2021 22:14:21 +0100 Subject: [PATCH 09/40] moar migration --- .../CogroupedStreamAggregateBuilder.java | 31 ++- .../GroupedStreamAggregateBuilder.java | 2 +- .../kstream/internals/KGroupedStreamImpl.java | 16 +- .../kstream/internals/KStreamAggregate.java | 62 +++-- .../KStreamAggregateProcessorSupplier.java | 27 +++ .../kstream/internals/KStreamReduce.java | 63 +++--- .../KStreamSessionWindowAggregate.java | 110 +++++---- .../KStreamSlidingWindowAggregate.java | 214 +++++++++--------- .../internals/KStreamWindowAggregate.java | 102 ++++----- .../kstream/internals/KTablePassThrough.java | 6 +- .../internals/SessionCacheFlushListener.java | 5 + .../SessionChangeCacheFlushListener.java | 48 ++++ .../internals/SessionRecordForwarder.java | 59 +++++ 13 files changed, 438 insertions(+), 307 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionRecordForwarder.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java index 1c5115d1dc639..29f631d4fbc79 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java @@ -38,7 +38,7 @@ 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.processor.api.ProcessorSupplier; import org.apache.kafka.streams.state.StoreBuilder; class CogroupedStreamAggregateBuilder { @@ -48,7 +48,6 @@ class CogroupedStreamAggregateBuilder { CogroupedStreamAggregateBuilder(final InternalStreamsBuilder builder) { this.builder = builder; } - @SuppressWarnings("unchecked") KTable build(final Map, Aggregator> groupPatterns, final Initializer initializer, final NamedInternal named, @@ -58,11 +57,11 @@ KTable build(final Map, Aggregator processors = new ArrayList<>(); - final Collection parentProcessors = new ArrayList<>(); + final Collection parentProcessors = new ArrayList<>(); boolean stateCreated = false; int counter = 0; for (final Entry, Aggregator> kGroupedStream : groupPatterns.entrySet()) { - final KStreamAggProcessorSupplier parentProcessor = + final KStreamAggregateProcessorSupplier parentProcessor = new KStreamAggregate<>(storeBuilder.name(), initializer, kGroupedStream.getValue()); parentProcessors.add(parentProcessor); final StatefulProcessorNode statefulProcessorNode = getStatefulProcessorNode( @@ -80,7 +79,6 @@ KTable build(final Map, Aggregator KTable build(final Map, Aggregator> groupPatterns, final Initializer initializer, final NamedInternal named, @@ -92,12 +90,12 @@ KTable build(final Map processRepartitions(groupPatterns, storeBuilder); final Collection processors = new ArrayList<>(); - final Collection parentProcessors = new ArrayList<>(); + final Collection parentProcessors = new ArrayList<>(); boolean stateCreated = false; int counter = 0; for (final Entry, Aggregator> kGroupedStream : groupPatterns.entrySet()) { - final KStreamAggProcessorSupplier parentProcessor = - (KStreamAggProcessorSupplier) new KStreamWindowAggregate( + final KStreamWindowAggregate parentProcessor = + new KStreamWindowAggregate<>( windows, storeBuilder.name(), initializer, @@ -118,7 +116,6 @@ KTable build(final Map return createTable(processors, parentProcessors, named, keySerde, valueSerde, queryableName, storeBuilder.name()); } - @SuppressWarnings("unchecked") KTable build(final Map, Aggregator> groupPatterns, final Initializer initializer, final NamedInternal named, @@ -130,12 +127,12 @@ KTable build(final Map, Aggregator sessionMerger) { processRepartitions(groupPatterns, storeBuilder); final Collection processors = new ArrayList<>(); - final Collection parentProcessors = new ArrayList<>(); + final Collection parentProcessors = new ArrayList<>(); boolean stateCreated = false; int counter = 0; for (final Entry, Aggregator> kGroupedStream : groupPatterns.entrySet()) { - final KStreamAggProcessorSupplier parentProcessor = - (KStreamAggProcessorSupplier) new KStreamSessionWindowAggregate( + final KStreamSessionWindowAggregate parentProcessor = + new KStreamSessionWindowAggregate<>( sessionWindows, storeBuilder.name(), initializer, @@ -167,13 +164,13 @@ KTable build(final Map, Aggregator parentProcessors = new ArrayList<>(); + final Collection parentProcessors = new ArrayList<>(); final Collection processors = new ArrayList<>(); boolean stateCreated = false; int counter = 0; for (final Entry, Aggregator> kGroupedStream : groupPatterns.entrySet()) { - final KStreamAggProcessorSupplier parentProcessor = - (KStreamAggProcessorSupplier) new KStreamSlidingWindowAggregate( + final KStreamSlidingWindowAggregate parentProcessor = + new KStreamSlidingWindowAggregate<>( slidingWindows, storeBuilder.name(), initializer, @@ -226,7 +223,7 @@ private void processRepartitions(final Map, Aggregator< @SuppressWarnings("unchecked") KTable createTable(final Collection processors, - final Collection parentProcessors, + final Collection parentProcessors, final NamedInternal named, final Serde keySerde, final Serde valueSerde, @@ -258,7 +255,7 @@ KTable createTable(final Collection processors, private StatefulProcessorNode getStatefulProcessorNode(final String processorName, final boolean stateCreated, final StoreBuilder storeBuilder, - final ProcessorSupplier kStreamAggregate) { + final ProcessorSupplier kStreamAggregate) { final StatefulProcessorNode statefulProcessorNode; if (!stateCreated) { statefulProcessorNode = diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java index 4896d97c64393..0dacd1345be33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java @@ -68,7 +68,7 @@ class GroupedStreamAggregateBuilder { KTable build(final NamedInternal functionName, final StoreBuilder storeBuilder, - final KStreamAggProcessorSupplier aggregateSupplier, + final KStreamAggregateProcessorSupplier aggregateSupplier, final String queryableStoreName, final Serde keySerde, final Serde valueSerde) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java index 13179969719a9..0c7d8cc48dcff 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.kstream.internals; +import java.util.Objects; +import java.util.Set; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.Aggregator; @@ -35,9 +37,6 @@ import org.apache.kafka.streams.kstream.internals.graph.GraphNode; import org.apache.kafka.streams.state.KeyValueStore; -import java.util.Objects; -import java.util.Set; - class KGroupedStreamImpl extends AbstractStream implements KGroupedStream { static final String REDUCE_NAME = "KSTREAM-REDUCE-"; @@ -183,7 +182,9 @@ private KTable doCount(final Named named, final Materialized(materializedInternal.storeName(), aggregateBuilder.countInitializer, aggregateBuilder.countAggregator), + new KStreamAggregate<>(materializedInternal.storeName(), + aggregateBuilder.countInitializer, + aggregateBuilder.countAggregator), name, materializedInternal); } @@ -233,9 +234,10 @@ public SessionWindowedKStream windowedBy(final SessionWindows windows) { ); } - private KTable doAggregate(final KStreamAggProcessorSupplier aggregateSupplier, - final String functionName, - final MaterializedInternal> materializedInternal) { + private KTable doAggregate( + final KStreamAggregateProcessorSupplier aggregateSupplier, + final String functionName, + final MaterializedInternal> materializedInternal) { return aggregateBuilder.build( new NamedInternal(functionName), new TimestampedKeyValueStoreMaterializer<>(materializedInternal).materialize(), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index 6a568d5fe89e2..ed7338eb680cd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -19,9 +19,9 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -31,7 +31,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamAggregate implements KStreamAggProcessorSupplier { +public class KStreamAggregate implements KStreamAggregateProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamAggregate.class); private final String storeName; private final Initializer initializer; @@ -48,7 +48,7 @@ public class KStreamAggregate implements KStreamAggProcessorSupplier get() { + public Processor> get() { return new KStreamAggregateProcessor(); } @@ -58,40 +58,38 @@ public void enableSendingOldValues() { } - private class KStreamAggregateProcessor extends AbstractProcessor { + private class KStreamAggregateProcessor implements Processor> { private TimestampedKeyValueStore store; private Sensor droppedRecordsSensor; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - super.init(context); + public void init(final ProcessorContext> context) { droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), context.taskId().toString(), (StreamsMetricsImpl) context.metrics()); - store = (TimestampedKeyValueStore) context.getStateStore(storeName); - tupleForwarder = new TimestampedTupleForwarder<>( + store = context.getStateStore(storeName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } @Override - public void process(final K key, final V value) { + public void process(Record record) { // If the key or value is null we don't need to proceed - if (key == null || value == null) { - LOG.warn( - "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", - key, value, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null || record.value() == null) { +// LOG.warn( +// "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// key, value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } - final ValueAndTimestamp oldAggAndTimestamp = store.get(key); + final ValueAndTimestamp oldAggAndTimestamp = store.get(record.key()); T oldAgg = getValueOrNull(oldAggAndTimestamp); final T newAgg; @@ -99,24 +97,24 @@ key, value, context().topic(), context().partition(), context().offset() if (oldAgg == null) { oldAgg = initializer.apply(); - newTimestamp = context().timestamp(); + newTimestamp = record.timestamp(); } else { oldAgg = oldAggAndTimestamp.value(); - newTimestamp = Math.max(context().timestamp(), oldAggAndTimestamp.timestamp()); + newTimestamp = Math.max(record.timestamp(), oldAggAndTimestamp.timestamp()); } - newAgg = aggregator.apply(key, value, oldAgg); + newAgg = aggregator.apply(record.key(), record.value(), oldAgg); - store.put(key, ValueAndTimestamp.make(newAgg, newTimestamp)); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, newTimestamp); + store.put(record.key(), ValueAndTimestamp.make(newAgg, newTimestamp)); + tupleForwarder.maybeForward(record, newAgg, sendOldValues ? oldAgg : null, newTimestamp); } } @Override - public KTableValueGetterSupplier view() { - return new KTableValueGetterSupplier() { + public KTableValueAndTimestampGetterSupplier view() { + return new KTableValueAndTimestampGetterSupplier() { - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KStreamAggregateValueGetter(); } @@ -128,13 +126,13 @@ public String[] storeNames() { } - private class KStreamAggregateValueGetter implements KTableValueGetter { + private class KStreamAggregateValueGetter implements KTableValueAndTimestampGetter { private TimestampedKeyValueStore store; - @SuppressWarnings("unchecked") + @Override - public void init(final ProcessorContext context) { - store = (TimestampedKeyValueStore) context.getStateStore(storeName); + public void init(ProcessorContext context) { + store = context.getStateStore(storeName); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java new file mode 100644 index 0000000000000..ee0364a867f60 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java @@ -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.kstream.internals; + +import org.apache.kafka.streams.processor.api.ProcessorSupplier; + +public interface KStreamAggregateProcessorSupplier extends ProcessorSupplier> { + + KTableValueAndTimestampGetterSupplier view(); + + void enableSendingOldValues(); +} + diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index 6c6923611fa33..df90e3cdeae9b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -18,9 +18,9 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -30,7 +30,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamReduce implements KStreamAggProcessorSupplier { +public class KStreamReduce implements KStreamAggregateProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamReduce.class); private final String storeName; @@ -44,7 +44,7 @@ public class KStreamReduce implements KStreamAggProcessorSupplier get() { + public Processor> get() { return new KStreamReduceProcessor(); } @@ -54,62 +54,60 @@ public void enableSendingOldValues() { } - private class KStreamReduceProcessor extends AbstractProcessor { + private class KStreamReduceProcessor implements Processor> { private TimestampedKeyValueStore store; - private TimestampedTupleForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - super.init(context); + public void init(final ProcessorContext> context) { metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); - store = (TimestampedKeyValueStore) context.getStateStore(storeName); - tupleForwarder = new TimestampedTupleForwarder<>( + store = context.getStateStore(storeName); + tupleForwarder = new TupleChangeForwarder<>( store, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } @Override - public void process(final K key, final V value) { + public void process(Record record) { // If the key or value is null we don't need to proceed - if (key == null || value == null) { - LOG.warn( - "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", - key, value, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null || record.value() == null) { +// LOG.warn( +// "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// key, value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } - final ValueAndTimestamp oldAggAndTimestamp = store.get(key); + final ValueAndTimestamp oldAggAndTimestamp = store.get(record.key()); final V oldAgg = getValueOrNull(oldAggAndTimestamp); final V newAgg; final long newTimestamp; if (oldAgg == null) { - newAgg = value; - newTimestamp = context().timestamp(); + newAgg = record.value(); + newTimestamp = record.timestamp(); } else { - newAgg = reducer.apply(oldAgg, value); - newTimestamp = Math.max(context().timestamp(), oldAggAndTimestamp.timestamp()); + newAgg = reducer.apply(oldAgg, record.value()); + newTimestamp = Math.max(record.timestamp(), oldAggAndTimestamp.timestamp()); } - store.put(key, ValueAndTimestamp.make(newAgg, newTimestamp)); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, newTimestamp); + store.put(record.key(), ValueAndTimestamp.make(newAgg, newTimestamp)); + tupleForwarder.maybeForward(record, newAgg, sendOldValues ? oldAgg : null, newTimestamp); } } @Override - public KTableValueGetterSupplier view() { - return new KTableValueGetterSupplier() { + public KTableValueAndTimestampGetterSupplier view() { + return new KTableValueAndTimestampGetterSupplier() { - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KStreamReduceValueGetter(); } @@ -121,13 +119,12 @@ public String[] storeNames() { } - private class KStreamReduceValueGetter implements KTableValueGetter { + private class KStreamReduceValueGetter implements KTableValueAndTimestampGetter { private TimestampedKeyValueStore store; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - store = (TimestampedKeyValueStore) context.getStateStore(storeName); + public void init(ProcessorContext context) { + store = context.getStateStore(storeName); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index 5648e8f0a367b..7eea3e95c6b5d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -24,9 +24,9 @@ import org.apache.kafka.streams.kstream.Merger; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; @@ -41,7 +41,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor; import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; -public class KStreamSessionWindowAggregate implements KStreamAggProcessorSupplier, V, Agg> { +public class KStreamSessionWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { private static final Logger LOG = LoggerFactory.getLogger(KStreamSessionWindowAggregate.class); private final String storeName; @@ -65,7 +65,7 @@ public KStreamSessionWindowAggregate(final SessionWindows windows, } @Override - public Processor get() { + public Processor, Change> get() { return new KStreamSessionWindowAggregateProcessor(); } @@ -78,21 +78,19 @@ public void enableSendingOldValues() { sendOldValues = true; } - private class KStreamSessionWindowAggregateProcessor extends AbstractProcessor { + private class KStreamSessionWindowAggregateProcessor implements Processor, Change> { private SessionStore store; - private SessionTupleForwarder tupleForwarder; + private SessionRecordForwarder tupleForwarder; private StreamsMetricsImpl metrics; - private InternalProcessorContext internalProcessorContext; private Sensor lateRecordDropSensor; private Sensor droppedRecordsSensor; private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - super.init(context); - internalProcessorContext = (InternalProcessorContext) context; + public void init(final ProcessorContext, Change> context) { + InternalProcessorContext, Change> internalProcessorContext = + (InternalProcessorContext, Change>) context; metrics = (StreamsMetricsImpl) context.metrics(); final String threadId = Thread.currentThread().getName(); lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor( @@ -102,24 +100,24 @@ public void init(final ProcessorContext context) { metrics ); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics); - store = (SessionStore) context.getStateStore(storeName); - tupleForwarder = new SessionTupleForwarder<>(store, context, new SessionCacheFlushListener<>(context), sendOldValues); + store = context.getStateStore(storeName); + tupleForwarder = new SessionRecordForwarder<>(store, context, new SessionCacheFlushListener<>(context), sendOldValues); } @Override - public void process(final K key, final V value) { + public void process(Record record) { // if the key is null, we do not need proceed aggregating // the record with the table - if (key == null) { - LOG.warn( - "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - value, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null) { +// LOG.warn( +// "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } - final long timestamp = context().timestamp(); + final long timestamp = record.timestamp(); observedStreamTime = Math.max(observedStreamTime, timestamp); final long closeTime = observedStreamTime - windows.gracePeriodMs(); @@ -130,7 +128,7 @@ value, context().topic(), context().partition(), context().offset() try ( final KeyValueIterator, Agg> iterator = store.findSessions( - key, + record.key(), timestamp - windows.inactivityGap(), timestamp + windows.inactivityGap() ) @@ -138,60 +136,60 @@ value, context().topic(), context().partition(), context().offset() while (iterator.hasNext()) { final KeyValue, Agg> next = iterator.next(); merged.add(next); - agg = sessionMerger.apply(key, agg, next.value); + agg = sessionMerger.apply(record.key(), agg, next.value); mergedWindow = mergeSessionWindow(mergedWindow, (SessionWindow) next.key.window()); } } if (mergedWindow.end() < closeTime) { - LOG.warn( - "Skipping record for expired window. " + - "key=[{}] " + - "topic=[{}] " + - "partition=[{}] " + - "offset=[{}] " + - "timestamp=[{}] " + - "window=[{},{}] " + - "expiration=[{}] " + - "streamTime=[{}]", - key, - context().topic(), - context().partition(), - context().offset(), - timestamp, - mergedWindow.start(), - mergedWindow.end(), - closeTime, - observedStreamTime - ); +// LOG.warn( +// "Skipping record for expired window. " + +// "key=[{}] " + +// "topic=[{}] " + +// "partition=[{}] " + +// "offset=[{}] " + +// "timestamp=[{}] " + +// "window=[{},{}] " + +// "expiration=[{}] " + +// "streamTime=[{}]", +// key, +// context().topic(), +// context().partition(), +// context().offset(), +// timestamp, +// mergedWindow.start(), +// mergedWindow.end(), +// closeTime, +// observedStreamTime +// ); lateRecordDropSensor.record(); } else { if (!mergedWindow.equals(newSessionWindow)) { for (final KeyValue, Agg> session : merged) { store.remove(session.key); - tupleForwarder.maybeForward(session.key, null, sendOldValues ? session.value : null); + tupleForwarder.maybeForward(record.withKey(session.key), null, sendOldValues ? session.value : null); } } - agg = aggregator.apply(key, value, agg); - final Windowed sessionKey = new Windowed<>(key, mergedWindow); + agg = aggregator.apply(record.key(), record.value(), agg); + final Windowed sessionKey = new Windowed<>(record.key(), mergedWindow); store.put(sessionKey, agg); - tupleForwarder.maybeForward(sessionKey, agg, null); + tupleForwarder.maybeForward(record.withKey(sessionKey), agg, null); } } } private SessionWindow mergeSessionWindow(final SessionWindow one, final SessionWindow two) { - final long start = one.start() < two.start() ? one.start() : two.start(); - final long end = one.end() > two.end() ? one.end() : two.end(); + final long start = Math.min(one.start(), two.start()); + final long end = Math.max(one.end(), two.end()); return new SessionWindow(start, end); } @Override - public KTableValueGetterSupplier, Agg> view() { - return new KTableValueGetterSupplier, Agg>() { + public KTableValueAndTimestampGetterSupplier, Agg> view() { + return new KTableValueAndTimestampGetterSupplier, Agg>() { @Override - public KTableValueGetter, Agg> get() { + public KTableValueAndTimestampGetter, Agg> get() { return new KTableSessionWindowValueGetter(); } @@ -202,13 +200,13 @@ public String[] storeNames() { }; } - private class KTableSessionWindowValueGetter implements KTableValueGetter, Agg> { + private class KTableSessionWindowValueGetter + implements KTableValueAndTimestampGetter, Agg> { private SessionStore store; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - store = (SessionStore) context.getStateStore(storeName); + public void init(ProcessorContext context) { + store = context.getStateStore(storeName); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index 250ea022b3855..a6fc9f7fc987a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -24,9 +24,9 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.SlidingWindows; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; @@ -41,7 +41,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamSlidingWindowAggregate implements KStreamAggProcessorSupplier, V, Agg> { +public class KStreamSlidingWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { private final Logger log = LoggerFactory.getLogger(getClass()); private final String storeName; @@ -62,7 +62,7 @@ public KStreamSlidingWindowAggregate(final SlidingWindows windows, } @Override - public Processor get() { + public Processor, Change> get() { return new KStreamSlidingWindowAggregateProcessor(); } @@ -75,9 +75,9 @@ public void enableSendingOldValues() { sendOldValues = true; } - private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor { + private class KStreamSlidingWindowAggregateProcessor implements Processor, Change> { private TimestampedWindowStore windowStore; - private TimestampedTupleForwarder, Agg> tupleForwarder; + private TupleChangeForwarder, Agg> tupleForwarder; private StreamsMetricsImpl metrics; private InternalProcessorContext internalProcessorContext; private Sensor lateRecordDropSensor; @@ -85,10 +85,8 @@ private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor, Change> context) { internalProcessorContext = (InternalProcessorContext) context; metrics = internalProcessorContext.metrics(); final String threadId = Thread.currentThread().getName(); @@ -99,61 +97,62 @@ public void init(final ProcessorContext context) { metrics ); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics); - windowStore = (TimestampedWindowStore) context.getStateStore(storeName); - tupleForwarder = new TimestampedTupleForwarder<>( + windowStore = context.getStateStore(storeName); + tupleForwarder = new TupleChangeForwarder<>( windowStore, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } + @Override - public void process(final K key, final V value) { - if (key == null || value == null) { - log.warn( - "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - value, context().topic(), context().partition(), context().offset() - ); + public void process(Record record) { + if (record.key() == null || record.value() == null) { +// log.warn( +// "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } - final long inputRecordTimestamp = context().timestamp(); + final long inputRecordTimestamp = record.timestamp(); observedStreamTime = Math.max(observedStreamTime, inputRecordTimestamp); final long closeTime = observedStreamTime - windows.gracePeriodMs(); if (inputRecordTimestamp + 1L + windows.timeDifferenceMs() <= closeTime) { - log.warn( - "Skipping record for expired window. " + - "key=[{}] " + - "topic=[{}] " + - "partition=[{}] " + - "offset=[{}] " + - "timestamp=[{}] " + - "window=[{},{}] " + - "expiration=[{}] " + - "streamTime=[{}]", - key, - context().topic(), - context().partition(), - context().offset(), - context().timestamp(), - inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp, - closeTime, - observedStreamTime - ); +// log.warn( +// "Skipping record for expired window. " + +// "key=[{}] " + +// "topic=[{}] " + +// "partition=[{}] " + +// "offset=[{}] " + +// "timestamp=[{}] " + +// "window=[{},{}] " + +// "expiration=[{}] " + +// "streamTime=[{}]", +// key, +// context().topic(), +// context().partition(), +// context().offset(), +// context().timestamp(), +// inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp, +// closeTime, +// observedStreamTime +// ); lateRecordDropSensor.record(); return; } if (inputRecordTimestamp < windows.timeDifferenceMs()) { - processEarly(key, value, inputRecordTimestamp, closeTime); + processEarly(record, inputRecordTimestamp, closeTime); return; } if (reverseIteratorPossible == null) { try { - windowStore.backwardFetch(key, 0L, 0L); + windowStore.backwardFetch(record.key(), 0L, 0L); reverseIteratorPossible = true; log.debug("Sliding Windows aggregate using a reverse iterator"); } catch (final UnsupportedOperationException e) { @@ -163,13 +162,13 @@ value, context().topic(), context().partition(), context().offset() } if (reverseIteratorPossible) { - processReverse(key, value, inputRecordTimestamp, closeTime); + processReverse(record, inputRecordTimestamp, closeTime); } else { - processInOrder(key, value, inputRecordTimestamp, closeTime); + processInOrder(record, inputRecordTimestamp, closeTime); } } - public void processInOrder(final K key, final V value, final long inputRecordTimestamp, final long closeTime) { + public void processInOrder(final Record record, final long inputRecordTimestamp, final long closeTime) { final Set windowStartTimes = new HashSet<>(); @@ -185,8 +184,7 @@ public void processInOrder(final K key, final V value, final long inputRecordTim try ( final KeyValueIterator, ValueAndTimestamp> iterator = windowStore.fetch( - key, - key, + record.key(), record.key(), Math.max(0, inputRecordTimestamp - 2 * windows.timeDifferenceMs()), // add 1 to upper bound to catch the current record's right window, if it exists, without more calls to the store inputRecordTimestamp + 1) @@ -206,10 +204,12 @@ public void processInOrder(final K key, final V value, final long inputRecordTim if (windowMaxRecordTimestamp < inputRecordTimestamp) { previousRecordTimestamp = windowMaxRecordTimestamp; } - updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, + record, closeTime, inputRecordTimestamp); } else if (endTime > inputRecordTimestamp && startTime <= inputRecordTimestamp) { rightWinAgg = windowBeingProcessed.value; - updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, + record, closeTime, inputRecordTimestamp); } else if (startTime == inputRecordTimestamp + 1) { rightWinAlreadyCreated = true; } else { @@ -221,10 +221,10 @@ public void processInOrder(final K key, final V value, final long inputRecordTim } } } - createWindows(key, value, inputRecordTimestamp, closeTime, windowStartTimes, rightWinAgg, leftWinAgg, leftWinAlreadyCreated, rightWinAlreadyCreated, previousRecordTimestamp); + createWindows(record, inputRecordTimestamp, closeTime, windowStartTimes, rightWinAgg, leftWinAgg, leftWinAlreadyCreated, rightWinAlreadyCreated, previousRecordTimestamp); } - public void processReverse(final K key, final V value, final long inputRecordTimestamp, final long closeTime) { + public void processReverse(final Record record, final long inputRecordTimestamp, final long closeTime) { final Set windowStartTimes = new HashSet<>(); @@ -240,8 +240,8 @@ public void processReverse(final K key, final V value, final long inputRecordTim try ( final KeyValueIterator, ValueAndTimestamp> iterator = windowStore.backwardFetch( - key, - key, + record.key(), + record.key(), Math.max(0, inputRecordTimestamp - 2 * windows.timeDifferenceMs()), // add 1 to upper bound to catch the current record's right window, if it exists, without more calls to the store inputRecordTimestamp + 1) @@ -258,10 +258,12 @@ public void processReverse(final K key, final V value, final long inputRecordTim if (rightWinAgg == null) { rightWinAgg = windowBeingProcessed.value; } - updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, + record, closeTime, inputRecordTimestamp); } else if (endTime == inputRecordTimestamp) { leftWinAlreadyCreated = true; - updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, + record, closeTime, inputRecordTimestamp); if (windowMaxRecordTimestamp < inputRecordTimestamp) { previousRecordTimestamp = windowMaxRecordTimestamp; } else { @@ -280,7 +282,7 @@ public void processReverse(final K key, final V value, final long inputRecordTim } } } - createWindows(key, value, inputRecordTimestamp, closeTime, windowStartTimes, rightWinAgg, leftWinAgg, leftWinAlreadyCreated, rightWinAlreadyCreated, previousRecordTimestamp); + createWindows(record, inputRecordTimestamp, closeTime, windowStartTimes, rightWinAgg, leftWinAgg, leftWinAlreadyCreated, rightWinAlreadyCreated, previousRecordTimestamp); } /** @@ -288,7 +290,7 @@ public void processReverse(final K key, final V value, final long inputRecordTim * windows with negative start times, which is not supported. Instead, we will put them into the [0, timeDifferenceMs] * window as a "workaround", and we will update or create their right windows as new records come in later */ - private void processEarly(final K key, final V value, final long inputRecordTimestamp, final long closeTime) { + private void processEarly(final Record record, final long inputRecordTimestamp, final long closeTime) { if (inputRecordTimestamp < 0 || inputRecordTimestamp >= windows.timeDifferenceMs()) { log.error( "Early record for sliding windows must fall between fall between 0 <= inputRecordTimestamp. Timestamp {} does not fall between 0 <= {}", @@ -307,8 +309,8 @@ private void processEarly(final K key, final V value, final long inputRecordTime try ( final KeyValueIterator, ValueAndTimestamp> iterator = windowStore.fetch( - key, - key, + record.key(), + record.key(), 0, // add 1 to upper bound to catch the current record's right window, if it exists, without more calls to the store inputRecordTimestamp + 1) @@ -330,7 +332,8 @@ private void processEarly(final K key, final V value, final long inputRecordTime } else if (startTime <= inputRecordTimestamp) { rightWinAgg = windowBeingProcessed.value; - updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, + record, closeTime, inputRecordTimestamp); } else if (startTime == inputRecordTimestamp + 1) { rightWinAlreadyCreated = true; } else { @@ -355,28 +358,29 @@ private void processEarly(final K key, final V value, final long inputRecordTime } if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, inputRecordTimestamp)) { - createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, key); + createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, record); } //create the right window for the previous record if the previous record exists and the window hasn't already been created if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) { - createPreviousRecordRightWindow(previousRecordTimestamp + 1, inputRecordTimestamp, key, value, closeTime); + createPreviousRecordRightWindow(previousRecordTimestamp + 1, inputRecordTimestamp, + record, closeTime); } if (combinedWindow == null) { final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs()); final ValueAndTimestamp valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp); - updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(window, valueAndTime, record, closeTime, inputRecordTimestamp); } else { //update the combined window with the new aggregate - updateWindowAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(combinedWindow.key.window(), combinedWindow.value, + record, closeTime, inputRecordTimestamp); } } - private void createWindows(final K key, - final V value, + private void createWindows(final Record record, final long inputRecordTimestamp, final long closeTime, final Set windowStartTimes, @@ -389,7 +393,7 @@ private void createWindows(final K key, if (previousRecordTimestamp != null) { final long previousRightWinStart = previousRecordTimestamp + 1; if (previousRecordRightWindowDoesNotExistAndIsNotEmpty(windowStartTimes, previousRightWinStart, inputRecordTimestamp)) { - createPreviousRecordRightWindow(previousRightWinStart, inputRecordTimestamp, key, value, closeTime); + createPreviousRecordRightWindow(previousRightWinStart, inputRecordTimestamp, record, closeTime); } } @@ -402,25 +406,25 @@ private void createWindows(final K key, valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp); } final TimeWindow window = new TimeWindow(inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp); - updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(window, valueAndTime, record, closeTime, inputRecordTimestamp); } // create right window for new record, if necessary if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, inputRecordTimestamp)) { - createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, key); + createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, record); } } private void createCurrentRecordRightWindow(final long inputRecordTimestamp, final ValueAndTimestamp rightWinAgg, - final K key) { + final Record record) { final TimeWindow window = new TimeWindow(inputRecordTimestamp + 1, inputRecordTimestamp + 1 + windows.timeDifferenceMs()); windowStore.put( - key, + record.key(), rightWinAgg, window.start()); tupleForwarder.maybeForward( - new Windowed<>(key, window), + record.withKey(new Windowed<>(record.key(), window)), rightWinAgg.value(), null, rightWinAgg.timestamp()); @@ -428,12 +432,11 @@ private void createCurrentRecordRightWindow(final long inputRecordTimestamp, private void createPreviousRecordRightWindow(final long windowStart, final long inputRecordTimestamp, - final K key, - final V value, + final Record record, final long closeTime) { final TimeWindow window = new TimeWindow(windowStart, windowStart + windows.timeDifferenceMs()); final ValueAndTimestamp valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp); - updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp); + updateWindowAndForward(window, valueAndTime, record, closeTime, inputRecordTimestamp); } // checks if the previous record falls into the current records left window; if yes, the left window is not empty, otherwise it is empty @@ -455,8 +458,7 @@ private boolean rightWindowIsNotEmpty(final ValueAndTimestamp rightWinAgg, private void updateWindowAndForward(final Window window, final ValueAndTimestamp valueAndTime, - final K key, - final V value, + final Record record, final long closeTime, final long inputRecordTimestamp) { final long windowStart = window.start(); @@ -464,48 +466,48 @@ private void updateWindowAndForward(final Window window, if (windowEnd > closeTime) { //get aggregate from existing window final Agg oldAgg = getValueOrNull(valueAndTime); - final Agg newAgg = aggregator.apply(key, value, oldAgg); + final Agg newAgg = aggregator.apply(record.key(), record.value(), oldAgg); final long newTimestamp = oldAgg == null ? inputRecordTimestamp : Math.max(inputRecordTimestamp, valueAndTime.timestamp()); windowStore.put( - key, + record.key(), ValueAndTimestamp.make(newAgg, newTimestamp), windowStart); tupleForwarder.maybeForward( - new Windowed(key, window), + record.withKey(new Windowed<>(record.key(), window)), newAgg, sendOldValues ? oldAgg : null, newTimestamp); } else { - log.warn( - "Skipping record for expired window. " + - "key=[{}] " + - "topic=[{}] " + - "partition=[{}] " + - "offset=[{}] " + - "timestamp=[{}] " + - "window=[{},{}] " + - "expiration=[{}] " + - "streamTime=[{}]", - key, - context().topic(), - context().partition(), - context().offset(), - context().timestamp(), - windowStart, windowEnd, - closeTime, - observedStreamTime - ); +// log.warn( +// "Skipping record for expired window. " + +// "key=[{}] " + +// "topic=[{}] " + +// "partition=[{}] " + +// "offset=[{}] " + +// "timestamp=[{}] " + +// "window=[{},{}] " + +// "expiration=[{}] " + +// "streamTime=[{}]", +// key, +// context().topic(), +// context().partition(), +// context().offset(), +// context().timestamp(), +// windowStart, windowEnd, +// closeTime, +// observedStreamTime +// ); lateRecordDropSensor.record(); } } } @Override - public KTableValueGetterSupplier, Agg> view() { - return new KTableValueGetterSupplier, Agg>() { + public KTableValueAndTimestampGetterSupplier, Agg> view() { + return new KTableValueAndTimestampGetterSupplier, Agg>() { - public KTableValueGetter, Agg> get() { + public KTableValueAndTimestampGetter, Agg> get() { return new KStreamWindowAggregateValueGetter(); } @@ -516,16 +518,16 @@ public String[] storeNames() { }; } - private class KStreamWindowAggregateValueGetter implements KTableValueGetter, Agg> { + private class KStreamWindowAggregateValueGetter + implements KTableValueAndTimestampGetter, Agg> { private TimestampedWindowStore windowStore; - @SuppressWarnings("unchecked") + @Override - public void init(final ProcessorContext context) { - windowStore = (TimestampedWindowStore) context.getStateStore(storeName); + public void init(ProcessorContext context) { + windowStore = context.getStateStore(storeName); } - @SuppressWarnings("unchecked") @Override public ValueAndTimestamp get(final Windowed windowedKey) { final K key = windowedKey.key(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index e1aa84bb46488..40368e850d9af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -23,9 +23,9 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windows; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedWindowStore; @@ -39,7 +39,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamWindowAggregate implements KStreamAggProcessorSupplier, V, Agg> { +public class KStreamWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { private final Logger log = LoggerFactory.getLogger(getClass()); private final String storeName; @@ -60,7 +60,7 @@ public KStreamWindowAggregate(final Windows windows, } @Override - public Processor get() { + public Processor, Change> get() { return new KStreamWindowAggregateProcessor(); } @@ -74,19 +74,17 @@ public void enableSendingOldValues() { } - private class KStreamWindowAggregateProcessor extends AbstractProcessor { + private class KStreamWindowAggregateProcessor implements Processor, Change> { private TimestampedWindowStore windowStore; - private TimestampedTupleForwarder, Agg> tupleForwarder; + private TupleChangeForwarder, Agg> tupleForwarder; private StreamsMetricsImpl metrics; private InternalProcessorContext internalProcessorContext; private Sensor lateRecordDropSensor; private Sensor droppedRecordsSensor; private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - super.init(context); + public void init(final ProcessorContext, Change> context) { internalProcessorContext = (InternalProcessorContext) context; metrics = internalProcessorContext.metrics(); final String threadId = Thread.currentThread().getName(); @@ -97,27 +95,27 @@ public void init(final ProcessorContext context) { metrics ); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics); - windowStore = (TimestampedWindowStore) context.getStateStore(storeName); - tupleForwarder = new TimestampedTupleForwarder<>( + windowStore = context.getStateStore(storeName); + tupleForwarder = new TupleChangeForwarder<>( windowStore, context, - new TimestampedCacheFlushListener<>(context), + new TupleChangeCacheFlushListener<>(context), sendOldValues); } @Override - public void process(final K key, final V value) { - if (key == null) { - log.warn( - "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - value, context().topic(), context().partition(), context().offset() - ); + public void process(Record record) { + if (record.key() == null) { +// log.warn( +// "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } // first get the matching windows - final long timestamp = context().timestamp(); + final long timestamp = record.timestamp(); observedStreamTime = Math.max(observedStreamTime, timestamp); final long closeTime = observedStreamTime - windows.gracePeriodMs(); @@ -128,7 +126,7 @@ value, context().topic(), context().partition(), context().offset() final Long windowStart = entry.getKey(); final long windowEnd = entry.getValue().end(); if (windowEnd > closeTime) { - final ValueAndTimestamp oldAggAndTimestamp = windowStore.fetch(key, windowStart); + final ValueAndTimestamp oldAggAndTimestamp = windowStore.fetch(record.key(), windowStart); Agg oldAgg = getValueOrNull(oldAggAndTimestamp); final Agg newAgg; @@ -136,40 +134,40 @@ value, context().topic(), context().partition(), context().offset() if (oldAgg == null) { oldAgg = initializer.apply(); - newTimestamp = context().timestamp(); + newTimestamp = record.timestamp(); } else { - newTimestamp = Math.max(context().timestamp(), oldAggAndTimestamp.timestamp()); + newTimestamp = Math.max(record.timestamp(), oldAggAndTimestamp.timestamp()); } - newAgg = aggregator.apply(key, value, oldAgg); + newAgg = aggregator.apply(record.key(), record.value(), oldAgg); // update the store with the new value - windowStore.put(key, ValueAndTimestamp.make(newAgg, newTimestamp), windowStart); + windowStore.put(record.key(), ValueAndTimestamp.make(newAgg, newTimestamp), windowStart); tupleForwarder.maybeForward( - new Windowed<>(key, entry.getValue()), + record.withKey(new Windowed<>(record.key(), entry.getValue())), newAgg, sendOldValues ? oldAgg : null, newTimestamp); } else { - log.warn( - "Skipping record for expired window. " + - "key=[{}] " + - "topic=[{}] " + - "partition=[{}] " + - "offset=[{}] " + - "timestamp=[{}] " + - "window=[{},{}) " + - "expiration=[{}] " + - "streamTime=[{}]", - key, - context().topic(), - context().partition(), - context().offset(), - context().timestamp(), - windowStart, windowEnd, - closeTime, - observedStreamTime - ); +// log.warn( +// "Skipping record for expired window. " + +// "key=[{}] " + +// "topic=[{}] " + +// "partition=[{}] " + +// "offset=[{}] " + +// "timestamp=[{}] " + +// "window=[{},{}) " + +// "expiration=[{}] " + +// "streamTime=[{}]", +// key, +// context().topic(), +// context().partition(), +// context().offset(), +// context().timestamp(), +// windowStart, windowEnd, +// closeTime, +// observedStreamTime +// ); lateRecordDropSensor.record(); } } @@ -177,10 +175,10 @@ value, context().topic(), context().partition(), context().offset() } @Override - public KTableValueGetterSupplier, Agg> view() { - return new KTableValueGetterSupplier, Agg>() { + public KTableValueAndTimestampGetterSupplier, Agg> view() { + return new KTableValueAndTimestampGetterSupplier, Agg>() { - public KTableValueGetter, Agg> get() { + public KTableValueAndTimestampGetter, Agg> get() { return new KStreamWindowAggregateValueGetter(); } @@ -192,13 +190,13 @@ public String[] storeNames() { } - private class KStreamWindowAggregateValueGetter implements KTableValueGetter, Agg> { + private class KStreamWindowAggregateValueGetter implements KTableValueAndTimestampGetter, Agg> { private TimestampedWindowStore windowStore; - @SuppressWarnings("unchecked") + @Override - public void init(final ProcessorContext context) { - windowStore = (TimestampedWindowStore) context.getStateStore(storeName); + public void init(ProcessorContext context) { + windowStore = context.getStateStore(storeName); } @SuppressWarnings("unchecked") diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java index 2e9a5afa150ab..e2140df80ca33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java @@ -26,11 +26,11 @@ import java.util.Collection; public class KTablePassThrough implements KTableChangeProcessorSupplier { - private final Collection parents; //TODO change to aggregationprocessor + private final Collection parents; //TODO change to aggregationprocessor private final String storeName; - KTablePassThrough(final Collection parents, final String storeName) { + KTablePassThrough(final Collection parents, final String storeName) { this.parents = parents; this.storeName = storeName; } @@ -43,7 +43,7 @@ public Processor, K, Change> get() { @Override public boolean enableSendingOldValues(final boolean forceMaterialization) { // Aggregation requires materialization so we will always enable sending old values - for (final KStreamAggProcessorSupplier parent : parents) { + for (final KStreamAggregateProcessorSupplier parent : parents) { parent.enableSendingOldValues(); } return true; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java index f40fdfe70ea6f..ca2b717f8c9d9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java @@ -27,6 +27,11 @@ class SessionCacheFlushListener implements CacheFlushListener, private final InternalProcessorContext context; private final ProcessorNode myNode; + SessionCacheFlushListener(final org.apache.kafka.streams.processor.api.ProcessorContext, Change> context) { + this.context = (InternalProcessorContext) context; + myNode = this.context.currentNode(); + } + SessionCacheFlushListener(final ProcessorContext context) { this.context = (InternalProcessorContext) context; myNode = this.context.currentNode(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java new file mode 100644 index 0000000000000..01458c5bf0a00 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java @@ -0,0 +1,48 @@ +/* + * 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; + +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorNode; +import org.apache.kafka.streams.state.internals.CacheFlushListener; + +class SessionChangeCacheFlushListener implements CacheFlushListener, V> { + private final InternalProcessorContext context; + private final ProcessorNode myNode; + + SessionChangeCacheFlushListener(final ProcessorContext context) { + this.context = (InternalProcessorContext) context; + myNode = this.context.currentNode(); + } + + @Override + public void apply(final Windowed key, + final V newValue, + final V oldValue, + final long timestamp) { + final ProcessorNode prev = context.currentNode(); + context.setCurrentNode(myNode); + try { + context.forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(key.window().end())); + } finally { + context.setCurrentNode(prev); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionRecordForwarder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionRecordForwarder.java new file mode 100644 index 0000000000000..141a37d87d275 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionRecordForwarder.java @@ -0,0 +1,59 @@ +/* + * 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; + +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.state.internals.CacheFlushListener; +import org.apache.kafka.streams.state.internals.WrappedStateStore; + +/** + * This class is used to determine if a processor should forward values to child nodes. Forwarding + * by this class only occurs when caching is not enabled. If caching is enabled, forwarding occurs + * in the flush listener when the cached store flushes. + * + * @param + * @param + */ +class SessionRecordForwarder { + + private final ProcessorContext, Change> context; + private final boolean sendOldValues; + private final boolean cachingEnabled; + + @SuppressWarnings("unchecked") + SessionRecordForwarder(final StateStore store, + final ProcessorContext, Change> context, + final CacheFlushListener, V> flushListener, + final boolean sendOldValues) { + this.context = context; + this.sendOldValues = sendOldValues; + cachingEnabled = ((WrappedStateStore) store).setFlushListener(flushListener, sendOldValues); + } + + public void maybeForward(final Record, ?> record, + final V newValue, + final V oldValue) { + if (!cachingEnabled) { + context.forward( + record.withValue(new Change<>(newValue, sendOldValues ? oldValue : null)) + .withTimestamp(record.key().window().end())); + } + } +} From a94eb300dd8988150fa4361f84e31df186be72e8 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 8 Apr 2021 22:39:13 +0100 Subject: [PATCH 10/40] moar migration --- .../kstream/internals/KStreamImpl.java | 11 +-- .../kstream/internals/KStreamKTableJoin.java | 12 +-- .../internals/KStreamKTableJoinProcessor.java | 29 +++---- .../streams/kstream/internals/KTableImpl.java | 67 +++++++-------- ...scriptionJoinForeignProcessorSupplier.java | 82 +++++++++++-------- ...criptionResolverJoinProcessorSupplier.java | 40 +++++---- .../internals/graph/ProcessorParameters.java | 6 -- .../graph/StatefulProcessorNode.java | 3 +- 8 files changed, 126 insertions(+), 124 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 547aa55f2bccc..5e0e286c5cb1e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1249,16 +1249,17 @@ private KStream doStreamTableJoin(final KTable table, final NamedInternal renamed = new NamedInternal(joinedInternal.name()); final String name = renamed.orElseGenerateWithPrefix(builder, leftJoin ? LEFTJOIN_NAME : JOIN_NAME); - final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier = new KStreamKTableJoin<>( - ((KTableImpl) table).valueGetterSupplier(), + final KStreamKTableJoin processorSupplier = new KStreamKTableJoin<>( + ((KTableImpl) table).valueAndTimestampGetterSupplier(), joiner, leftJoin); - final ProcessorParameters processorParameters = new ProcessorParameters<>(processorSupplier, name); - final StreamTableJoinNode streamTableJoinNode = new StreamTableJoinNode<>( + final ProcessorParameters processorParameters = new ProcessorParameters<>( + processorSupplier, name); + final StreamTableJoinNode streamTableJoinNode = new StreamTableJoinNode<>( name, processorParameters, - ((KTableImpl) table).valueGetterSupplier().storeNames(), + ((KTableImpl) table).valueAndTimestampGetterSupplier().storeNames(), this.name ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java index 136b953441b29..bc85a884a6abd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java @@ -18,17 +18,17 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; -class KStreamKTableJoin implements ProcessorSupplier { +class KStreamKTableJoin implements ProcessorSupplier { private final KeyValueMapper keyValueMapper = (key, value) -> key; - private final KTableValueGetterSupplier valueGetterSupplier; + private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; private final ValueJoinerWithKey joiner; private final boolean leftJoin; - KStreamKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, + KStreamKTableJoin(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, final ValueJoinerWithKey joiner, final boolean leftJoin) { this.valueGetterSupplier = valueGetterSupplier; @@ -37,7 +37,7 @@ class KStreamKTableJoin implements ProcessorSupplier { } @Override - public Processor get() { + public Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), keyValueMapper, joiner, leftJoin); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 5cf31424b8fb2..7aab6139ebbfc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -19,8 +19,9 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,17 +29,17 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KStreamKTableJoinProcessor extends AbstractProcessor { +class KStreamKTableJoinProcessor extends ContextualProcessor { private static final Logger LOG = LoggerFactory.getLogger(KStreamKTableJoin.class); - private final KTableValueGetter valueGetter; + private final KTableValueAndTimestampGetter valueGetter; private final KeyValueMapper keyMapper; private final ValueJoinerWithKey joiner; private final boolean leftJoin; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KStreamKTableJoinProcessor(final KTableValueGetter valueGetter, + KStreamKTableJoinProcessor(final KTableValueAndTimestampGetter valueGetter, final KeyValueMapper keyMapper, final ValueJoinerWithKey joiner, final boolean leftJoin) { @@ -49,7 +50,7 @@ class KStreamKTableJoinProcessor extends AbstractProcessor context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -57,7 +58,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K1 key, final V1 value) { + public void process(final Record record) { // we do join iff the join keys are equal, thus, if {@code keyMapper} returns {@code null} we // cannot join and just ignore the record. Note for KTables, this is the same as having a null key // since keyMapper just returns the key, but for GlobalKTables we can have other keyMappers @@ -66,17 +67,17 @@ public void process(final K1 key, final V1 value) { // an empty message (ie, there is nothing to be joined) -- this contrast SQL NULL semantics // furthermore, on left/outer joins 'null' in ValueJoiner#apply() indicates a missing record -- // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored - final K2 mappedKey = keyMapper.apply(key, value); - if (mappedKey == null || value == null) { - LOG.warn( - "Skipping record due to null join key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", - key, value, context().topic(), context().partition(), context().offset() - ); + final K2 mappedKey = keyMapper.apply(record.key(), record.value()); + if (mappedKey == null || record.value() == null) { +// LOG.warn( +// "Skipping record due to null join key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// key, value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); } else { final V2 value2 = getValueOrNull(valueGetter.get(mappedKey)); if (leftJoin || value2 != null) { - context().forward(key, joiner.apply(key, value, value2)); + context().forward(record.withValue(joiner.apply(record.key(), record.value(), value2))); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 0f0e086355726..02c22b01548b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -16,6 +16,16 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.apache.kafka.streams.kstream.internals.graph.GraphGraceSearchUtil.findAndVerifyWindowGrace; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; @@ -46,13 +56,13 @@ import org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionStoreReceiveProcessorSupplier; import org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionWrapper; import org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionWrapperSerde; +import org.apache.kafka.streams.kstream.internals.graph.GraphNode; import org.apache.kafka.streams.kstream.internals.graph.KTableKTableJoinNode; import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode; 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.StreamSinkNode; import org.apache.kafka.streams.kstream.internals.graph.StreamSourceNode; -import org.apache.kafka.streams.kstream.internals.graph.GraphNode; import org.apache.kafka.streams.kstream.internals.graph.TableProcessorNode; import org.apache.kafka.streams.kstream.internals.suppress.FinalResultsSuppressionBuilder; import org.apache.kafka.streams.kstream.internals.suppress.KTableSuppressProcessorSupplier; @@ -70,17 +80,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.Duration; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.function.Function; -import java.util.function.Supplier; - -import static org.apache.kafka.streams.kstream.internals.graph.GraphGraceSearchUtil.findAndVerifyWindowGrace; - /** * The implementation class of {@link KTable}. * @@ -754,8 +753,9 @@ private KTable doJoin(final KTable other, .withJoinOtherProcessorParameters(joinOtherProcessorParameters) .withThisJoinSideNodeName(name) .withOtherJoinSideNodeName(((KTableImpl) other).name) - .withJoinThisStoreNames(valueGetterSupplier().storeNames()) - .withJoinOtherStoreNames(((KTableImpl) other).valueGetterSupplier().storeNames()) + .withJoinThisStoreNames(valueAndTimestampGetterSupplier().storeNames()) + .withJoinOtherStoreNames( + ((KTableImpl) other).valueAndTimestampGetterSupplier().storeNames()) .withKeySerde(keySerde) .withValueSerde(valueSerde) .withQueryableStoreName(queryableStoreName) @@ -820,20 +820,6 @@ public KTableValueAndTimestampGetterSupplier valueAndTimestampGetterSuppli } } - @SuppressWarnings("unchecked") - public KTableValueGetterSupplier valueGetterSupplier() { - if (processorSupplier instanceof KTableSource) { - final KTableSource source = (KTableSource) processorSupplier; - // whenever a source ktable is required for getter, it should be materialized - source.materialize(); - return new KTableSourceValueGetterSupplier<>(source.queryableName()); - } else if (processorSupplier instanceof KStreamAggProcessorSupplier) { - return ((KStreamAggProcessorSupplier) processorSupplier).view(); - } else { - return ((KTableProcessorSupplier) processorSupplier).view(); - } - } - @SuppressWarnings("unchecked") public boolean enableSendingOldValues(final boolean forceMaterialization) { if (!sendOldValues) { @@ -1080,12 +1066,13 @@ private KTable doJoinOnForeignKey(final KTable forei new StatefulProcessorNode<>( new ProcessorParameters<>( new SubscriptionJoinForeignProcessorSupplier<>( - ((KTableImpl) foreignKeyTable).valueGetterSupplier() + ((KTableImpl) foreignKeyTable).valueAndTimestampGetterSupplier() ), renamed.suffixWithOrElseGet("-subscription-join-foreign", builder, SUBSCRIPTION_PROCESSOR) ), Collections.emptySet(), - Collections.singleton(((KTableImpl) foreignKeyTable).valueGetterSupplier()) + Collections.singleton( + ((KTableImpl) foreignKeyTable).valueAndTimestampGetterSupplier()) ); builder.addGraphNode(subscriptionReceiveNode, subscriptionJoinForeignNode); @@ -1124,18 +1111,20 @@ private KTable doJoinOnForeignKey(final KTable forei resultSourceNodes.add(foreignResponseSource.nodeName()); builder.internalTopologyBuilder.copartitionSources(resultSourceNodes); - final KTableValueGetterSupplier primaryKeyValueGetter = valueGetterSupplier(); - final SubscriptionResolverJoinProcessorSupplier resolverProcessorSupplier = new SubscriptionResolverJoinProcessorSupplier<>( - primaryKeyValueGetter, - valueSerde == null ? null : valueSerde.serializer(), - valueHashSerdePseudoTopic, - joiner, - leftJoin - ); + final KTableValueAndTimestampGetterSupplier primaryKeyValueGetter = valueAndTimestampGetterSupplier(); + final SubscriptionResolverJoinProcessorSupplier resolverProcessorSupplier = + new SubscriptionResolverJoinProcessorSupplier<>( + primaryKeyValueGetter, + valueSerde == null ? null : valueSerde.serializer(), + valueHashSerdePseudoTopic, + joiner, + leftJoin + ); final StatefulProcessorNode> resolverNode = new StatefulProcessorNode<>( new ProcessorParameters<>( resolverProcessorSupplier, - renamed.suffixWithOrElseGet("-subscription-response-resolver", builder, SUBSCRIPTION_RESPONSE_RESOLVER_PROCESSOR) + renamed.suffixWithOrElseGet("-subscription-response-resolver", builder, + SUBSCRIPTION_RESPONSE_RESOLVER_PROCESSOR) ), Collections.emptySet(), Collections.singleton(primaryKeyValueGetter) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java index 2544eb1856b7c..be5fe70924e18 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java @@ -17,67 +17,75 @@ package org.apache.kafka.streams.kstream.internals.foreignkeyjoin; +import java.util.Objects; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.kstream.internals.KTableValueGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.ValueAndTimestamp; -import java.util.Objects; - /** * Receives {@code SubscriptionWrapper} events and processes them according to their Instruction. - * Depending on the results, {@code SubscriptionResponseWrapper}s are created, which will be propagated to - * the {@code SubscriptionResolverJoinProcessorSupplier} instance. + * Depending on the results, {@code SubscriptionResponseWrapper}s are created, which will be + * propagated to the {@code SubscriptionResolverJoinProcessorSupplier} instance. * - * @param Type of primary keys + * @param Type of primary keys * @param Type of foreign key * @param Type of foreign value */ public class SubscriptionJoinForeignProcessorSupplier - implements ProcessorSupplier, Change>>> { + implements + ProcessorSupplier, Change>>, CombinedKey, SubscriptionResponseWrapper> { - private final KTableValueGetterSupplier foreignValueGetterSupplier; + private final KTableValueAndTimestampGetterSupplier foreignValueGetterSupplier; - public SubscriptionJoinForeignProcessorSupplier(final KTableValueGetterSupplier foreignValueGetterSupplier) { + public SubscriptionJoinForeignProcessorSupplier( + final KTableValueAndTimestampGetterSupplier foreignValueGetterSupplier) { this.foreignValueGetterSupplier = foreignValueGetterSupplier; } @Override - public Processor, Change>>> get() { + public Processor, Change>>, CombinedKey, SubscriptionResponseWrapper> get() { - return new AbstractProcessor, Change>>>() { + return new ContextualProcessor, Change>>, CombinedKey, SubscriptionResponseWrapper>() { - private KTableValueGetter foreignValues; + private KTableValueAndTimestampGetter foreignValues; @Override - public void init(final ProcessorContext context) { + public void init( + final ProcessorContext, SubscriptionResponseWrapper> context) { super.init(context); foreignValues = foreignValueGetterSupplier.get(); foreignValues.init(context); } @Override - public void process(final CombinedKey combinedKey, final Change>> change) { - Objects.requireNonNull(combinedKey, "This processor should never see a null key."); - Objects.requireNonNull(change, "This processor should never see a null value."); - final ValueAndTimestamp> valueAndTimestamp = change.newValue; - Objects.requireNonNull(valueAndTimestamp, "This processor should never see a null newValue."); + public void process( + Record, Change>>> record) { + Objects.requireNonNull(record.key(), "This processor should never see a null key."); + Objects.requireNonNull(record.value(), + "This processor should never see a null value."); + final ValueAndTimestamp> valueAndTimestamp = record + .value().newValue; + Objects.requireNonNull(valueAndTimestamp, + "This processor should never see a null newValue."); final SubscriptionWrapper value = valueAndTimestamp.value(); if (value.getVersion() != SubscriptionWrapper.CURRENT_VERSION) { //Guard against modifications to SubscriptionWrapper. Need to ensure that there is compatibility //with previous versions to enable rolling upgrades. Must develop a strategy for upgrading //from older SubscriptionWrapper versions to newer versions. - throw new UnsupportedVersionException("SubscriptionWrapper is of an incompatible version."); + throw new UnsupportedVersionException( + "SubscriptionWrapper is of an incompatible version."); } - final ValueAndTimestamp foreignValueAndTime = foreignValues.get(combinedKey.getForeignKey()); + final ValueAndTimestamp foreignValueAndTime = foreignValues + .get(record.key().getForeignKey()); final long resultTimestamp = foreignValueAndTime == null ? @@ -87,9 +95,10 @@ public void process(final CombinedKey combinedKey, final Change(value.getHash(), null), - To.all().withTimestamp(resultTimestamp) + record + .withValue( + new SubscriptionResponseWrapper(value.getHash(), null)) + .withTimestamp(resultTimestamp) ); break; case PROPAGATE_NULL_IF_NO_FK_VAL_AVAILABLE: @@ -99,17 +108,20 @@ public void process(final CombinedKey combinedKey, final Change(value.getHash(), valueToSend), - To.all().withTimestamp(resultTimestamp) + record + .withValue( + new SubscriptionResponseWrapper<>(value.getHash(), valueToSend)) + .withTimestamp(resultTimestamp) ); break; case PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE: if (foreignValueAndTime != null) { context().forward( - combinedKey.getPrimaryKey(), - new SubscriptionResponseWrapper<>(value.getHash(), foreignValueAndTime.value()), - To.all().withTimestamp(resultTimestamp) + record + .withValue( + new SubscriptionResponseWrapper<>(value.getHash(), + foreignValueAndTime.value())) + .withTimestamp(resultTimestamp) ); } break; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java index 3cd06368a7131..084bacef55de5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java @@ -20,12 +20,16 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; import org.apache.kafka.streams.kstream.internals.KTableValueGetter; import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.Murmur3; @@ -41,14 +45,14 @@ * @param Type of foreign values * @param Type of joined result of primary and foreign values */ -public class SubscriptionResolverJoinProcessorSupplier implements ProcessorSupplier> { - private final KTableValueGetterSupplier valueGetterSupplier; +public class SubscriptionResolverJoinProcessorSupplier implements ProcessorSupplier, K, VR> { + private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; private final Serializer constructionTimeValueSerializer; private final Supplier valueHashSerdePseudoTopicSupplier; private final ValueJoiner joiner; private final boolean leftJoin; - public SubscriptionResolverJoinProcessorSupplier(final KTableValueGetterSupplier valueGetterSupplier, + public SubscriptionResolverJoinProcessorSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, final Serializer valueSerializer, final Supplier valueHashSerdePseudoTopicSupplier, final ValueJoiner joiner, @@ -61,16 +65,16 @@ public SubscriptionResolverJoinProcessorSupplier(final KTableValueGetterSupplier } @Override - public Processor> get() { - return new AbstractProcessor>() { + public Processor, K, VR> get() { + return new ContextualProcessor, K, VR>() { private String valueHashSerdePseudoTopic; private Serializer runtimeValueSerializer = constructionTimeValueSerializer; - private KTableValueGetter valueGetter; + private KTableValueAndTimestampGetter valueGetter; @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); valueHashSerdePseudoTopic = valueHashSerdePseudoTopicSupplier.get(); valueGetter = valueGetterSupplier.get(); @@ -81,31 +85,31 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K key, final SubscriptionResponseWrapper value) { - if (value.getVersion() != SubscriptionResponseWrapper.CURRENT_VERSION) { + public void process(Record> record) { + if (record.value().getVersion() != SubscriptionResponseWrapper.CURRENT_VERSION) { //Guard against modifications to SubscriptionResponseWrapper. Need to ensure that there is //compatibility with previous versions to enable rolling upgrades. Must develop a strategy for //upgrading from older SubscriptionWrapper versions to newer versions. throw new UnsupportedVersionException("SubscriptionResponseWrapper is of an incompatible version."); } - final ValueAndTimestamp currentValueWithTimestamp = valueGetter.get(key); + final ValueAndTimestamp currentValueWithTimestamp = valueGetter.get(record.key()); final long[] currentHash = currentValueWithTimestamp == null ? null : Murmur3.hash128(runtimeValueSerializer.serialize(valueHashSerdePseudoTopic, currentValueWithTimestamp.value())); - final long[] messageHash = value.getOriginalValueHash(); + final long[] messageHash = record.value().getOriginalValueHash(); //If this value doesn't match the current value from the original table, it is stale and should be discarded. if (java.util.Arrays.equals(messageHash, currentHash)) { final VR result; - if (value.getForeignValue() == null && (!leftJoin || currentValueWithTimestamp == null)) { + if (record.value().getForeignValue() == null && (!leftJoin || currentValueWithTimestamp == null)) { result = null; //Emit tombstone } else { - result = joiner.apply(currentValueWithTimestamp == null ? null : currentValueWithTimestamp.value(), value.getForeignValue()); + result = joiner.apply(currentValueWithTimestamp == null ? null : currentValueWithTimestamp.value(), record.value().getForeignValue()); } - context().forward(key, result); + context().forward(record.withValue(result)); } } }; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java index 51c40d9b3990a..8860a8f44ada7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java @@ -19,7 +19,6 @@ import org.apache.kafka.streams.kstream.internals.KTableChangeProcessorSupplier; import org.apache.kafka.streams.kstream.internals.KTableKTableJoinMerger; -import org.apache.kafka.streams.kstream.internals.KTableProcessorSupplier; import org.apache.kafka.streams.kstream.internals.KTableSource; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.ProcessorAdapter; @@ -71,11 +70,6 @@ KTableSource kTableSourceSupplier() { : (KTableSource) oldProcessorSupplier; } - KTableProcessorSupplier kTableProcessorSupplier() { - // This cast always works because KTableProcessorSupplier hasn't been converted yet. - return (KTableProcessorSupplier) oldProcessorSupplier; - } - KTableChangeProcessorSupplier kTableChangeProcessorSupplier() { // This cast always works because KTableProcessorSupplier hasn't been converted yet. return (KTableChangeProcessorSupplier) processorSupplier; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java index 28af95b6dd68c..e3d22fa37562f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals.graph; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; @@ -35,7 +36,7 @@ public class StatefulProcessorNode extends ProcessorGraphNode { */ public StatefulProcessorNode(final ProcessorParameters processorParameters, final Set> preRegisteredStores, - final Set> valueGetterSuppliers) { + final Set> valueGetterSuppliers) { super(processorParameters.processorName(), processorParameters); final Stream registeredStoreNames = preRegisteredStores.stream().map(StoreBuilder::name); final Stream valueGetterStoreNames = valueGetterSuppliers.stream().flatMap(s -> Arrays.stream(s.storeNames())); From ea67f6882bf3ba897faf9d748d09ba981abb26fc Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 8 Apr 2021 23:01:13 +0100 Subject: [PATCH 11/40] migrate table joins --- .../internals/KTableKTableAbstractJoin.java | 10 +-- ...KTableAbstractJoinValueGetterSupplier.java | 10 +-- .../internals/KTableKTableInnerJoin.java | 85 ++++++++++--------- .../internals/KTableKTableLeftJoin.java | 67 ++++++++------- .../internals/KTableKTableOuterJoin.java | 67 ++++++++------- .../internals/KTableKTableRightJoin.java | 60 ++++++------- 6 files changed, 154 insertions(+), 145 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java index ecaaf459100b7..4d693218fb4b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java @@ -18,12 +18,12 @@ import org.apache.kafka.streams.kstream.ValueJoiner; -abstract class KTableKTableAbstractJoin implements KTableProcessorSupplier { +abstract class KTableKTableAbstractJoin implements KTableChangeProcessorSupplier { private final KTableImpl table1; private final KTableImpl table2; - final KTableValueGetterSupplier valueGetterSupplier1; - final KTableValueGetterSupplier valueGetterSupplier2; + final KTableValueAndTimestampGetterSupplier valueGetterSupplier1; + final KTableValueAndTimestampGetterSupplier valueGetterSupplier2; final ValueJoiner joiner; boolean sendOldValues = false; @@ -33,8 +33,8 @@ abstract class KTableKTableAbstractJoin implements KTableProcessor final ValueJoiner joiner) { this.table1 = table1; this.table2 = table2; - this.valueGetterSupplier1 = table1.valueGetterSupplier(); - this.valueGetterSupplier2 = table2.valueGetterSupplier(); + this.valueGetterSupplier1 = table1.valueAndTimestampGetterSupplier(); + this.valueGetterSupplier2 = table2.valueAndTimestampGetterSupplier(); this.joiner = joiner; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java index 924452ddfd3f6..cb0b5fa47b5e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java @@ -20,12 +20,12 @@ import java.util.HashSet; import java.util.Set; -public abstract class KTableKTableAbstractJoinValueGetterSupplier implements KTableValueGetterSupplier { - final KTableValueGetterSupplier valueGetterSupplier1; - final KTableValueGetterSupplier valueGetterSupplier2; +public abstract class KTableKTableAbstractJoinValueGetterSupplier implements KTableValueAndTimestampGetterSupplier { + final KTableValueAndTimestampGetterSupplier valueGetterSupplier1; + final KTableValueAndTimestampGetterSupplier valueGetterSupplier2; - KTableKTableAbstractJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableAbstractJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, + final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { this.valueGetterSupplier1 = valueGetterSupplier1; this.valueGetterSupplier2 = valueGetterSupplier2; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 06701c2756d68..d4c9f109dc813 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -16,21 +16,21 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; +import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; + import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; -import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; - class KTableKTableInnerJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableInnerJoin.class); @@ -43,39 +43,43 @@ class KTableKTableInnerJoin extends KTableKTableAbstractJoin> get() { + public Processor, K, Change> get() { return new KTableKTableJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { - return new KTableKTableInnerJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); + public KTableValueAndTimestampGetterSupplier view() { + return new KTableKTableInnerJoinValueGetterSupplier(valueGetterSupplier1, + valueGetterSupplier2); } - private class KTableKTableInnerJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableInnerJoinValueGetterSupplier extends + KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableInnerJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableInnerJoinValueGetterSupplier( + final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, + final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { - return new KTableKTableInnerJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); + public KTableValueAndTimestampGetter get() { + return new KTableKTableInnerJoinValueGetter(valueGetterSupplier1.get(), + valueGetterSupplier2.get()); } } - private class KTableKTableJoinProcessor extends AbstractProcessor> { + private class KTableKTableJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueAndTimestampGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { this.valueGetter = valueGetter; } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -83,13 +87,13 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K key, final Change change) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record - if (key == null) { - LOG.warn( - "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", - change, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null) { +// LOG.warn( +// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", +// change, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } @@ -98,23 +102,24 @@ change, context().topic(), context().partition(), context().offset() final long resultTimestamp; R oldValue = null; - final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(key); + final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(record.key()); final V2 valueRight = getValueOrNull(valueAndTimestampRight); if (valueRight == null) { return; } - resultTimestamp = Math.max(context().timestamp(), valueAndTimestampRight.timestamp()); + resultTimestamp = Math.max(record.timestamp(), valueAndTimestampRight.timestamp()); - if (change.newValue != null) { - newValue = joiner.apply(change.newValue, valueRight); + if (record.value().newValue != null) { + newValue = joiner.apply(record.value().newValue, valueRight); } - if (sendOldValues && change.oldValue != null) { - oldValue = joiner.apply(change.oldValue, valueRight); + if (sendOldValues && record.value().oldValue != null) { + oldValue = joiner.apply(record.value().oldValue, valueRight); } - context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); + context().forward(record.withValue(new Change<>(newValue, oldValue)) + .withTimestamp(resultTimestamp)); } @Override @@ -123,21 +128,22 @@ public void close() { } } - private class KTableKTableInnerJoinValueGetter implements KTableValueGetter { + private class KTableKTableInnerJoinValueGetter implements KTableValueAndTimestampGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueAndTimestampGetter valueGetter1; + private final KTableValueAndTimestampGetter valueGetter2; - KTableKTableInnerJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableInnerJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, + final KTableValueAndTimestampGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); + } @Override @@ -146,7 +152,8 @@ public ValueAndTimestamp get(final K key) { final V1 value1 = getValueOrNull(valueAndTimestamp1); if (value1 != null) { - final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(keyValueMapper.apply(key, value1)); + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2 + .get(keyValueMapper.apply(key, value1)); final V2 value2 = getValueOrNull(valueAndTimestamp2); if (value2 != null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index b6fd894ae3db5..295f7cc089870 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -18,10 +18,10 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; @@ -41,40 +41,40 @@ class KTableKTableLeftJoin extends KTableKTableAbstractJoin> get() { + public Processor, K, Change> get() { return new KTableKTableLeftJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { return new KTableKTableLeftJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } private class KTableKTableLeftJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableLeftJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableLeftJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, + final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KTableKTableLeftJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } - private class KTableKTableLeftJoinProcessor extends AbstractProcessor> { + private class KTableKTableLeftJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueAndTimestampGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableLeftJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableLeftJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { this.valueGetter = valueGetter; } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -82,13 +82,13 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K key, final Change change) { + public void process(Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record - if (key == null) { - LOG.warn( - "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", - change, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null) { +// LOG.warn( +// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", +// change, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } @@ -97,12 +97,12 @@ change, context().topic(), context().partition(), context().offset() final long resultTimestamp; R oldValue = null; - final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(key); + final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(record.key()); final V2 value2 = getValueOrNull(valueAndTimestampRight); final long timestampRight; if (value2 == null) { - if (change.newValue == null && change.oldValue == null) { + if (record.value().newValue == null && record.value().oldValue == null) { return; } timestampRight = UNKNOWN; @@ -110,17 +110,18 @@ change, context().topic(), context().partition(), context().offset() timestampRight = valueAndTimestampRight.timestamp(); } - resultTimestamp = Math.max(context().timestamp(), timestampRight); + resultTimestamp = Math.max(record.timestamp(), timestampRight); - if (change.newValue != null) { - newValue = joiner.apply(change.newValue, value2); + if (record.value().newValue != null) { + newValue = joiner.apply(record.value().newValue, value2); } - if (sendOldValues && change.oldValue != null) { - oldValue = joiner.apply(change.oldValue, value2); + if (sendOldValues && record.value().oldValue != null) { + oldValue = joiner.apply(record.value().oldValue, value2); } - context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); + context().forward(record.withValue(new Change<>(newValue, oldValue)) + .withTimestamp(resultTimestamp)); } @Override @@ -129,19 +130,19 @@ public void close() { } } - private class KTableKTableLeftJoinValueGetter implements KTableValueGetter { + private class KTableKTableLeftJoinValueGetter implements KTableValueAndTimestampGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueAndTimestampGetter valueGetter1; + private final KTableValueAndTimestampGetter valueGetter2; - KTableKTableLeftJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableLeftJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, + final KTableValueAndTimestampGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 391255d2d030c..6280412d65e7a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -18,10 +18,10 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; @@ -41,34 +41,34 @@ class KTableKTableOuterJoin extends KTableKTableAbstractJoin> get() { + public Processor, K, Change> get() { return new KTableKTableOuterJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { return new KTableKTableOuterJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } private class KTableKTableOuterJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableOuterJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableOuterJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, + final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KTableKTableOuterJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } - private class KTableKTableOuterJoinProcessor extends AbstractProcessor> { + private class KTableKTableOuterJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueAndTimestampGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableOuterJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableOuterJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { this.valueGetter = valueGetter; } @@ -81,13 +81,13 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K key, final Change change) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record - if (key == null) { - LOG.warn( - "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", - change, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null) { +// LOG.warn( +// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", +// change, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } @@ -96,26 +96,27 @@ change, context().topic(), context().partition(), context().offset() final long resultTimestamp; R oldValue = null; - final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(key); + final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(record.key()); final V2 value2 = getValueOrNull(valueAndTimestamp2); if (value2 == null) { - if (change.newValue == null && change.oldValue == null) { + if (record.value().newValue == null && record.value().oldValue == null) { return; } - resultTimestamp = context().timestamp(); + resultTimestamp = record.timestamp(); } else { - resultTimestamp = Math.max(context().timestamp(), valueAndTimestamp2.timestamp()); + resultTimestamp = Math.max(record.timestamp(), valueAndTimestamp2.timestamp()); } - if (value2 != null || change.newValue != null) { - newValue = joiner.apply(change.newValue, value2); + if (value2 != null || record.value().newValue != null) { + newValue = joiner.apply(record.value().newValue, value2); } - if (sendOldValues && (value2 != null || change.oldValue != null)) { - oldValue = joiner.apply(change.oldValue, value2); + if (sendOldValues && (value2 != null || record.value().oldValue != null)) { + oldValue = joiner.apply(record.value().oldValue, value2); } - context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); + context().forward(record.withValue(new Change<>(newValue, oldValue)) + .withTimestamp(resultTimestamp)); } @Override @@ -124,19 +125,19 @@ public void close() { } } - private class KTableKTableOuterJoinValueGetter implements KTableValueGetter { + private class KTableKTableOuterJoinValueGetter implements KTableValueAndTimestampGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueAndTimestampGetter valueGetter1; + private final KTableValueAndTimestampGetter valueGetter2; - KTableKTableOuterJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableOuterJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, + final KTableValueAndTimestampGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index f499c2f89b43c..eba22c87fc0f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -18,10 +18,10 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; @@ -40,39 +40,39 @@ class KTableKTableRightJoin extends KTableKTableAbstractJoin> get() { + public Processor, K, Change> get() { return new KTableKTableRightJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueAndTimestampGetterSupplier view() { return new KTableKTableRightJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } private class KTableKTableRightJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableRightJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableRightJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, + final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { + public KTableValueAndTimestampGetter get() { return new KTableKTableRightJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } - private class KTableKTableRightJoinProcessor extends AbstractProcessor> { + private class KTableKTableRightJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueAndTimestampGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableRightJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableRightJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { this.valueGetter = valueGetter; } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -80,13 +80,13 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K key, final Change change) { + public void process(Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record - if (key == null) { - LOG.warn( - "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", - change, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null) { +// LOG.warn( +// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", +// change, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } @@ -95,23 +95,23 @@ change, context().topic(), context().partition(), context().offset() final long resultTimestamp; R oldValue = null; - final ValueAndTimestamp valueAndTimestampLeft = valueGetter.get(key); + final ValueAndTimestamp valueAndTimestampLeft = valueGetter.get(record.key()); final V2 valueLeft = getValueOrNull(valueAndTimestampLeft); if (valueLeft == null) { return; } - resultTimestamp = Math.max(context().timestamp(), valueAndTimestampLeft.timestamp()); + resultTimestamp = Math.max(record.timestamp(), valueAndTimestampLeft.timestamp()); // joiner == "reverse joiner" - newValue = joiner.apply(change.newValue, valueLeft); + newValue = joiner.apply(record.value().newValue, valueLeft); if (sendOldValues) { // joiner == "reverse joiner" - oldValue = joiner.apply(change.oldValue, valueLeft); + oldValue = joiner.apply(record.value().oldValue, valueLeft); } - context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); + context().forward(record.withValue(new Change<>(newValue, oldValue)).withTimestamp(resultTimestamp)); } @Override @@ -120,19 +120,19 @@ public void close() { } } - private class KTableKTableRightJoinValueGetter implements KTableValueGetter { + private class KTableKTableRightJoinValueGetter implements KTableValueAndTimestampGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueAndTimestampGetter valueGetter1; + private final KTableValueAndTimestampGetter valueGetter2; - KTableKTableRightJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableRightJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, + final KTableValueAndTimestampGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); } From c7be1c2ddf6d14d3d8843eb808d5836564ec2b4c Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 8 Apr 2021 23:15:21 +0100 Subject: [PATCH 12/40] migrate global ktable --- .../kstream/internals/GlobalKTableImpl.java | 6 +-- .../internals/KStreamGlobalKTableJoin.java | 12 +++--- .../kstream/internals/KStreamImpl.java | 37 ++++++++++--------- 3 files changed, 28 insertions(+), 27 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java index 734ff4a17e711..10f2658c27d35 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java @@ -20,16 +20,16 @@ public class GlobalKTableImpl implements GlobalKTable { - private final KTableValueGetterSupplier valueGetterSupplier; + private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; private final String queryableStoreName; - GlobalKTableImpl(final KTableValueGetterSupplier valueGetterSupplier, + GlobalKTableImpl(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, final String queryableStoreName) { this.valueGetterSupplier = valueGetterSupplier; this.queryableStoreName = queryableStoreName; } - KTableValueGetterSupplier valueGetterSupplier() { + KTableValueAndTimestampGetterSupplier valueGetterSupplier() { return valueGetterSupplier; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java index 568a639efeb53..9156ddd29a655 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java @@ -18,17 +18,17 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; -class KStreamGlobalKTableJoin implements ProcessorSupplier { +class KStreamGlobalKTableJoin implements ProcessorSupplier { - private final KTableValueGetterSupplier valueGetterSupplier; + private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; private final ValueJoinerWithKey joiner; private final KeyValueMapper mapper; private final boolean leftJoin; - KStreamGlobalKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, + KStreamGlobalKTableJoin(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, final ValueJoinerWithKey joiner, final KeyValueMapper mapper, final boolean leftJoin) { @@ -39,7 +39,7 @@ class KStreamGlobalKTableJoin implements ProcessorSupplier get() { + public Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), mapper, joiner, leftJoin); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 5e0e286c5cb1e..8e5b8c04c70e4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -16,12 +16,21 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode.optimizableRepartitionNodeBuilder; + +import java.lang.reflect.Array; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.BranchedKStream; import org.apache.kafka.streams.kstream.ForeachAction; +import org.apache.kafka.streams.kstream.ForeachProcessor; import org.apache.kafka.streams.kstream.GlobalKTable; import org.apache.kafka.streams.kstream.Grouped; import org.apache.kafka.streams.kstream.JoinWindows; @@ -46,6 +55,7 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.graph.BaseRepartitionNode; import org.apache.kafka.streams.kstream.internals.graph.BaseRepartitionNode.BaseRepartitionNodeBuilder; +import org.apache.kafka.streams.kstream.internals.graph.GraphNode; import org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode; import org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode.OptimizableRepartitionNodeBuilder; import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode; @@ -54,27 +64,16 @@ import org.apache.kafka.streams.kstream.internals.graph.StreamSinkNode; import org.apache.kafka.streams.kstream.internals.graph.StreamTableJoinNode; import org.apache.kafka.streams.kstream.internals.graph.StreamToTableNode; -import org.apache.kafka.streams.kstream.internals.graph.GraphNode; import org.apache.kafka.streams.kstream.internals.graph.UnoptimizableRepartitionNode; import org.apache.kafka.streams.kstream.internals.graph.UnoptimizableRepartitionNode.UnoptimizableRepartitionNodeBuilder; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; -import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopicNameExtractor; -import org.apache.kafka.streams.kstream.ForeachProcessor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopicProperties; import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor; import org.apache.kafka.streams.state.KeyValueStore; -import java.lang.reflect.Array; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.Objects; -import java.util.Set; - -import static org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode.optimizableRepartitionNodeBuilder; - public class KStreamImpl extends AbstractStream implements KStream { static final String JOINTHIS_NAME = "KSTREAM-JOINTHIS-"; @@ -1210,17 +1209,19 @@ private KStream globalTableJoin(final GlobalKTable g Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(named, "named can't be null"); - final KTableValueGetterSupplier valueGetterSupplier = + final KTableValueAndTimestampGetterSupplier valueGetterSupplier = ((GlobalKTableImpl) globalTable).valueGetterSupplier(); - final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, LEFTJOIN_NAME); - final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( + final String name = new NamedInternal(named) + .orElseGenerateWithPrefix(builder, LEFTJOIN_NAME); + final ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( valueGetterSupplier, joiner, keySelector, leftJoin); - final ProcessorParameters processorParameters = new ProcessorParameters<>(processorSupplier, name); - final StreamTableJoinNode streamTableJoinNode = - new StreamTableJoinNode<>(name, processorParameters, new String[] {}, null); + final ProcessorParameters processorParameters = new ProcessorParameters<>( + processorSupplier, name); + final StreamTableJoinNode streamTableJoinNode = new StreamTableJoinNode<>( + name, processorParameters, new String[] {}, null); builder.addGraphNode(graphNode, streamTableJoinNode); From 0c3cbc956d0d3fa59f60a96532084aa82ff70d5c Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 8 Apr 2021 23:25:21 +0100 Subject: [PATCH 13/40] migrate foreign key --- .../foreignkeyjoin/CombinedKeySchema.java | 13 ++++- ...eignJoinSubscriptionProcessorSupplier.java | 54 +++++++++++-------- ...criptionStoreReceiveProcessorSupplier.java | 50 ++++++++--------- 3 files changed, 69 insertions(+), 48 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java index 57bc646a13e11..edefd1f4dcdd9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import java.nio.ByteBuffer; import java.util.function.Supplier; @@ -51,7 +51,16 @@ public CombinedKeySchema(final Supplier foreignKeySerdeTopicSupplier, } @SuppressWarnings("unchecked") - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { + primaryKeySerdeTopic = undecoratedPrimaryKeySerdeTopicSupplier.get(); + foreignKeySerdeTopic = undecoratedForeignKeySerdeTopicSupplier.get(); + primaryKeySerializer = primaryKeySerializer == null ? (Serializer) context.keySerde().serializer() : primaryKeySerializer; + primaryKeyDeserializer = primaryKeyDeserializer == null ? (Deserializer) context.keySerde().deserializer() : primaryKeyDeserializer; + foreignKeySerializer = foreignKeySerializer == null ? (Serializer) context.keySerde().serializer() : foreignKeySerializer; + foreignKeyDeserializer = foreignKeyDeserializer == null ? (Deserializer) context.keySerde().deserializer() : foreignKeyDeserializer; + } + + public void init(final ProcessorContext context) { primaryKeySerdeTopic = undecoratedPrimaryKeySerdeTopicSupplier.get(); foreignKeySerdeTopic = undecoratedForeignKeySerdeTopicSupplier.get(); primaryKeySerializer = primaryKeySerializer == null ? (Serializer) context.keySerde().serializer() : primaryKeySerializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java index fd95105324472..cd950fe8e0e45 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java @@ -17,15 +17,17 @@ package org.apache.kafka.streams.kstream.internals.foreignkeyjoin; +import java.nio.ByteBuffer; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.KeyValueIterator; @@ -35,10 +37,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; +public class ForeignJoinSubscriptionProcessorSupplier implements + ProcessorSupplier, K, SubscriptionResponseWrapper> { -public class ForeignJoinSubscriptionProcessorSupplier implements ProcessorSupplier> { - private static final Logger LOG = LoggerFactory.getLogger(ForeignJoinSubscriptionProcessorSupplier.class); + private static final Logger LOG = LoggerFactory + .getLogger(ForeignJoinSubscriptionProcessorSupplier.class); private final StoreBuilder>> storeBuilder; private final CombinedKeySchema keySchema; @@ -51,19 +54,22 @@ public ForeignJoinSubscriptionProcessorSupplier( } @Override - public Processor> get() { + public Processor, K, SubscriptionResponseWrapper> get() { return new KTableKTableJoinProcessor(); } - private final class KTableKTableJoinProcessor extends AbstractProcessor> { + private final class KTableKTableJoinProcessor extends + ContextualProcessor, K, SubscriptionResponseWrapper> { + private Sensor droppedRecordsSensor; private TimestampedKeyValueStore> store; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); - final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; + final InternalProcessorContext> internalProcessorContext = + (InternalProcessorContext>) context; droppedRecordsSensor = TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), internalProcessorContext.taskId().toString(), @@ -76,32 +82,36 @@ public void init(final ProcessorContext context) { * @throws StreamsException if key is null */ @Override - public void process(final KO key, final Change value) { + public void process(Record> record) { // if the key is null, we do not need proceed aggregating // the record with the table - if (key == null) { - LOG.warn( - "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - value, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null) { +// LOG.warn( +// "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } - final Bytes prefixBytes = keySchema.prefixBytes(key); + final Bytes prefixBytes = keySchema.prefixBytes(record.key()); //Perform the prefixScan and propagate the results try (final KeyValueIterator>> prefixScanResults = - store.range(prefixBytes, Bytes.increment(prefixBytes))) { + store.range(prefixBytes, Bytes.increment(prefixBytes))) { while (prefixScanResults.hasNext()) { - final KeyValue>> next = prefixScanResults.next(); + final KeyValue>> next = prefixScanResults + .next(); // have to check the prefix because the range end is inclusive :( if (prefixEquals(next.key.get(), prefixBytes.get())) { final CombinedKey combinedKey = keySchema.fromBytes(next.key); context().forward( - combinedKey.getPrimaryKey(), - new SubscriptionResponseWrapper<>(next.value.value().getHash(), value.newValue) + record.withKey( + combinedKey.getPrimaryKey()) + .withValue( + new SubscriptionResponseWrapper<>(next.value.value().getHash(), + record.value().newValue)) ); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java index 61fb1c1334774..fe17caad2ce3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java @@ -21,11 +21,11 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.StoreBuilder; @@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory; public class SubscriptionStoreReceiveProcessorSupplier - implements ProcessorSupplier> { + implements ProcessorSupplier, CombinedKey, Change>>> { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionStoreReceiveProcessorSupplier.class); private final StoreBuilder>> storeBuilder; @@ -50,17 +50,18 @@ public SubscriptionStoreReceiveProcessorSupplier( } @Override - public Processor> get() { + public Processor, CombinedKey, Change>>> get() { - return new AbstractProcessor>() { + return new ContextualProcessor, CombinedKey, Change>>>() { private TimestampedKeyValueStore> store; private Sensor droppedRecordsSensor; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext, Change>>> context) { super.init(context); - final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; + final InternalProcessorContext, Change>>> internalProcessorContext = + (InternalProcessorContext, Change>>>) context; droppedRecordsSensor = TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), @@ -73,30 +74,30 @@ public void init(final ProcessorContext context) { } @Override - public void process(final KO key, final SubscriptionWrapper value) { - if (key == null) { - LOG.warn( - "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - value, context().topic(), context().partition(), context().offset() - ); + public void process(Record> record) { + if (record.key() == null) { +// LOG.warn( +// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } - if (value.getVersion() != SubscriptionWrapper.CURRENT_VERSION) { + if (record.value().getVersion() != SubscriptionWrapper.CURRENT_VERSION) { //Guard against modifications to SubscriptionWrapper. Need to ensure that there is compatibility //with previous versions to enable rolling upgrades. Must develop a strategy for upgrading //from older SubscriptionWrapper versions to newer versions. throw new UnsupportedVersionException("SubscriptionWrapper is of an incompatible version."); } - final Bytes subscriptionKey = keySchema.toBytes(key, value.getPrimaryKey()); + final Bytes subscriptionKey = keySchema.toBytes(record.key(), record.value().getPrimaryKey()); - final ValueAndTimestamp> newValue = ValueAndTimestamp.make(value, context().timestamp()); + final ValueAndTimestamp> newValue = ValueAndTimestamp.make(record.value(), record.timestamp()); final ValueAndTimestamp> oldValue = store.get(subscriptionKey); //This store is used by the prefix scanner in ForeignJoinSubscriptionProcessorSupplier - if (value.getInstruction().equals(SubscriptionWrapper.Instruction.DELETE_KEY_AND_PROPAGATE) || - value.getInstruction().equals(SubscriptionWrapper.Instruction.DELETE_KEY_NO_PROPAGATE)) { + if (record.value().getInstruction().equals(SubscriptionWrapper.Instruction.DELETE_KEY_AND_PROPAGATE) || + record.value().getInstruction().equals(SubscriptionWrapper.Instruction.DELETE_KEY_NO_PROPAGATE)) { store.delete(subscriptionKey); } else { store.put(subscriptionKey, newValue); @@ -105,9 +106,10 @@ value, context().topic(), context().partition(), context().offset() // note: key is non-nullable // note: newValue is non-nullable context().forward( - new CombinedKey<>(key, value.getPrimaryKey()), - change, - To.all().withTimestamp(newValue.timestamp()) + record.withKey( + new CombinedKey<>(record.key(), record.value().getPrimaryKey())) + .withValue(change) + .withTimestamp(newValue.timestamp()) ); } }; From 8baf291f43e2551e835fbfffeace605b2bf8cf1d Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 8 Apr 2021 23:42:46 +0100 Subject: [PATCH 14/40] migrate ktable repartition map --- .../internals/InternalStreamsBuilder.java | 2 +- .../streams/kstream/internals/KTableImpl.java | 2 +- .../internals/KTableRepartitionMap.java | 75 +++++++++++-------- 3 files changed, 46 insertions(+), 33 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java index 6fa65c13894d4..514dfbec80849 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java @@ -185,7 +185,7 @@ public GlobalKTable globalTable(final String topic, addGraphNode(root, tableSourceNode); - return new GlobalKTableImpl<>(new KTableSourceValueGetterSupplier<>(storeName), materialized.queryableStoreName()); + return new GlobalKTableImpl<>(new KTableSourceValueAndTimestampGetterSupplier<>(storeName), materialized.queryableStoreName()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 02c22b01548b9..06db4225d2389 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -789,7 +789,7 @@ public KGroupedTable groupBy(final KeyValueMapper groupedInternal = new GroupedInternal<>(grouped); final String selectName = new NamedInternal(groupedInternal.name()).orElseGenerateWithPrefix(builder, SELECT_NAME); - final KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); + final KTableChangeProcessorSupplier, K1, V1> selectSupplier = new KTableRepartitionMap<>(this, selector); final ProcessorParameters, ?, ?> processorParameters = new ProcessorParameters<>(selectSupplier, selectName); // select the aggregate key and values (old and new), it would require parent to send old values diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index d5dc5db0f44d8..750ae056eadbb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -16,49 +16,56 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; + import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.ValueAndTimestamp; -import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; - /** - * KTable repartition map functions are not exposed to public APIs, but only used for keyed aggregations. + * KTable repartition map functions are not exposed to public APIs, but only used for keyed + * aggregations. *

- * Given the input, it can output at most two records (one mapped from old value and one mapped from new value). + * Given the input, it can output at most two records (one mapped from old value and one mapped from + * new value). */ -public class KTableRepartitionMap implements KTableProcessorSupplier> { +public class KTableRepartitionMap implements + KTableChangeProcessorSupplier, K1, V1> { private final KTableImpl parent; private final KeyValueMapper> mapper; - KTableRepartitionMap(final KTableImpl parent, final KeyValueMapper> mapper) { + KTableRepartitionMap(final KTableImpl parent, + final KeyValueMapper> mapper) { this.parent = parent; this.mapper = mapper; } @Override - public Processor> get() { + public Processor, K1, Change> get() { return new KTableMapProcessor(); } @Override - public KTableValueGetterSupplier> view() { - final KTableValueGetterSupplier parentValueGetterSupplier = parent.valueGetterSupplier(); + public KTableValueAndTimestampGetterSupplier> view() { + final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = + parent.valueAndTimestampGetterSupplier(); - return new KTableValueGetterSupplier>() { + return new KTableValueAndTimestampGetterSupplier>() { - public KTableValueGetter> get() { + public KTableValueAndTimestampGetter> get() { return new KTableMapValueGetter(parentValueGetterSupplier.get()); } @Override public String[] storeNames() { - throw new StreamsException("Underlying state store not accessible due to repartitioning."); + throw new StreamsException( + "Underlying state store not accessible due to repartitioning."); } }; } @@ -72,46 +79,51 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { throw new IllegalStateException("KTableRepartitionMap should always require sending old values."); } - private class KTableMapProcessor extends AbstractProcessor> { + private class KTableMapProcessor extends ContextualProcessor, K1, Change> { /** * @throws StreamsException if key is null */ @Override - public void process(final K key, final Change change) { + public void process(Record> record) { // the original key should never be null - if (key == null) { - throw new StreamsException("Record key for the grouping KTable should not be null."); + if (record.key() == null) { + throw new StreamsException( + "Record key for the grouping KTable should not be null."); } // if the value is null, we do not need to forward its selected key-value further - final KeyValue newPair = change.newValue == null ? null : mapper.apply(key, change.newValue); - final KeyValue oldPair = change.oldValue == null ? null : mapper.apply(key, change.oldValue); + final KeyValue newPair = + record.value().newValue == null ? null + : mapper.apply(record.key(), record.value().newValue); + final KeyValue oldPair = + record.value().oldValue == null ? null + : mapper.apply(record.key(), record.value().oldValue); // if the selected repartition key or value is null, skip // forward oldPair first, to be consistent with reduce and aggregate if (oldPair != null && oldPair.key != null && oldPair.value != null) { - context().forward(oldPair.key, new Change<>(null, oldPair.value)); + context().forward(record.withKey(oldPair.key) + .withValue(new Change<>(null, oldPair.value))); } if (newPair != null && newPair.key != null && newPair.value != null) { - context().forward(newPair.key, new Change<>(newPair.value, null)); + context().forward(record.withKey(newPair.key) + .withValue(new Change<>(newPair.value, null))); } - } } - private class KTableMapValueGetter implements KTableValueGetter> { - private final KTableValueGetter parentGetter; - private ProcessorContext context; + private class KTableMapValueGetter implements KTableValueAndTimestampGetter> { + private final KTableValueAndTimestampGetter parentGetter; - KTableMapValueGetter(final KTableValueGetter parentGetter) { + KTableMapValueGetter(final KTableValueAndTimestampGetter parentGetter) { this.parentGetter = parentGetter; } + @Override - public void init(final ProcessorContext context) { - this.context = context; + public void init(ProcessorContext context) { parentGetter.init(context); } @@ -120,7 +132,8 @@ public ValueAndTimestamp> get(final K key) { final ValueAndTimestamp valueAndTimestamp = parentGetter.get(key); return ValueAndTimestamp.make( mapper.apply(key, getValueOrNull(valueAndTimestamp)), - valueAndTimestamp == null ? context.timestamp() : valueAndTimestamp.timestamp()); + valueAndTimestamp.timestamp()); + //TODO valueAndTimestamp == null ? context.timestamp() : valueAndTimestamp.timestamp()); } @Override From 662a10c613fa424412d36e816b16082d3f5cc041 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 12:01:07 +0100 Subject: [PATCH 15/40] compile source --- .../apache/kafka/streams/kstream/KStream.java | 10 +-- .../internals/KStreamFlatTransform.java | 27 ++++---- .../internals/KStreamFlatTransformValues.java | 28 ++++---- .../kstream/internals/KStreamImpl.java | 52 +++++++------- .../kstream/internals/KStreamJoinWindow.java | 23 ++++--- .../kstream/internals/KStreamKStreamJoin.java | 55 ++++++++------- .../internals/KStreamTransformValues.java | 25 +++---- .../streams/kstream/internals/KTableImpl.java | 8 ++- .../internals/KTableKTableOuterJoin.java | 2 +- .../internals/SessionCacheFlushListener.java | 13 ++-- .../SessionChangeCacheFlushListener.java | 48 ------------- .../TimestampedCacheFlushListener.java | 53 --------------- .../internals/TimestampedTupleForwarder.java | 63 ----------------- .../internals/TupleChangeForwarder.java | 29 ++++---- .../foreignkeyjoin/CombinedKeySchema.java | 9 --- ...JoinSubscriptionSendProcessorSupplier.java | 67 ++++++++++--------- .../internals/graph/GraphGraceSearchUtil.java | 4 +- .../internals/graph/ProcessorParameters.java | 37 +++------- .../graph/StatefulProcessorNode.java | 4 +- .../internals/graph/StreamToTableNode.java | 4 +- .../internals/GlobalStateUpdateTask.java | 5 +- .../internals/InternalTopologyBuilder.java | 32 ++++----- .../processor/internals/ProcessorNode.java | 8 +-- .../internals/ProcessorTopology.java | 26 +++---- .../internals/RecordDeserializer.java | 6 +- .../processor/internals/RecordQueue.java | 6 +- .../streams/processor/internals/SinkNode.java | 4 +- .../processor/internals/SourceNode.java | 6 +- .../processor/internals/StreamTask.java | 7 +- .../kstream/internals/KStreamImplTest.java | 8 +-- .../TimestampedCacheFlushListenerTest.java | 8 +-- .../TimestampedTupleForwarderTest.java | 44 +++++++----- .../internals/GlobalStateTaskTest.java | 6 +- .../internals/ProcessorTopologyFactories.java | 2 +- .../internals/RecordDeserializerTest.java | 2 +- .../processor/internals/RecordQueueTest.java | 2 +- .../processor/internals/StreamTaskTest.java | 16 ++--- .../test/MockInternalProcessorContext.java | 2 +- .../apache/kafka/test/MockProcessorNode.java | 2 +- .../org/apache/kafka/test/MockSourceNode.java | 4 +- 40 files changed, 296 insertions(+), 461 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListener.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarder.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index e58114a83b37c..7a89ddc33732f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -4615,8 +4615,8 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier processorSupplier, - final String... stateStoreNames); +// void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, +// final String... stateStoreNames); /** * Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given @@ -4709,9 +4709,9 @@ void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, - final Named named, - final String... stateStoreNames); +// void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, +// final Named named, +// final String... stateStoreNames); void process(final ProcessorSupplier processorSupplier, final String... stateStoreNames); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java index 7836d440d372a..783acf557e1b1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java @@ -19,15 +19,16 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; -public class KStreamFlatTransform implements ProcessorSupplier { +public class KStreamFlatTransform implements ProcessorSupplier { private final TransformerSupplier>> transformerSupplier; @@ -36,7 +37,7 @@ public KStreamFlatTransform(final TransformerSupplier get() { + public Processor get() { return new KStreamFlatTransformProcessor<>(transformerSupplier.get()); } @@ -45,7 +46,8 @@ public Set> stores() { return transformerSupplier.stores(); } - public static class KStreamFlatTransformProcessor extends AbstractProcessor { + public static class KStreamFlatTransformProcessor extends + ContextualProcessor { private final Transformer>> transformer; @@ -54,17 +56,18 @@ public KStreamFlatTransformProcessor(final Transformer context) { super.init(context); - transformer.init(context); + //TODO transformer.init(context); } @Override - public void process(final KIn key, final VIn value) { - final Iterable> pairs = transformer.transform(key, value); + public void process(Record record) { + final Iterable> pairs = + transformer.transform(record.key(), record.value()); if (pairs != null) { for (final KeyValue pair : pairs) { - context().forward(pair.key, pair.value); + context().forward(record.withKey(pair.key).withValue(pair.value)); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java index 62955951bdf51..822c89d1bcbfa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java @@ -18,16 +18,16 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; -public class KStreamFlatTransformValues implements ProcessorSupplier { +public class KStreamFlatTransformValues implements ProcessorSupplier { private final ValueTransformerWithKeySupplier> valueTransformerSupplier; @@ -36,7 +36,7 @@ public KStreamFlatTransformValues(final ValueTransformerWithKeySupplier get() { + public Processor get() { return new KStreamFlatTransformValuesProcessor<>(valueTransformerSupplier.get()); } @@ -45,7 +45,8 @@ public Set> stores() { return valueTransformerSupplier.stores(); } - public static class KStreamFlatTransformValuesProcessor extends AbstractProcessor { + public static class KStreamFlatTransformValuesProcessor extends + ContextualProcessor { private final ValueTransformerWithKey> valueTransformer; @@ -54,17 +55,18 @@ public static class KStreamFlatTransformValuesProcessor extends } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + // TODO valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } @Override - public void process(final KIn key, final VIn value) { - final Iterable transformedValues = valueTransformer.transform(key, value); + public void process(Record record) { + final Iterable transformedValues = + valueTransformer.transform(record.key(), record.value()); if (transformedValues != null) { for (final VOut transformedValue : transformedValues) { - context.forward(key, transformedValue); + context.forward(record.withValue(transformedValue)); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 8e5b8c04c70e4..161810930ac5a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1465,32 +1465,32 @@ private KStream doFlatTransformValues(final ValueTransformerWithKeyS builder); } - @Override - public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, - final String... stateStoreNames) { - process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); - } - - @Override - public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, - final Named named, - final String... stateStoreNames) { - Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); - Objects.requireNonNull(named, "named can't be null"); - Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); - ApiUtils.checkSupplier(processorSupplier); - for (final String stateStoreName : stateStoreNames) { - Objects.requireNonNull(stateStoreName, "stateStoreNames can't be null"); - } - - final String name = new NamedInternal(named).name(); - final StatefulProcessorNode processNode = new StatefulProcessorNode<>( - name, - new ProcessorParameters<>(processorSupplier, name), - stateStoreNames); - - builder.addGraphNode(graphNode, processNode); - } +// @Override +// public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, +// final String... stateStoreNames) { +// process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); +// } +// +// @Override +// public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, +// final Named named, +// final String... stateStoreNames) { +// Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); +// Objects.requireNonNull(named, "named can't be null"); +// Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); +// ApiUtils.checkSupplier(processorSupplier); +// for (final String stateStoreName : stateStoreNames) { +// Objects.requireNonNull(stateStoreName, "stateStoreNames can't be null"); +// } +// +// final String name = new NamedInternal(named).name(); +// final StatefulProcessorNode processNode = new StatefulProcessorNode<>( +// name, +// new ProcessorParameters<>(processorSupplier, name), +// stateStoreNames); +// +// builder.addGraphNode(graphNode, processNode); +// } @Override public void process(final ProcessorSupplier processorSupplier, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index 34756d47cf2ee..c76b0cc168335 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -16,13 +16,14 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.WindowStore; -class KStreamJoinWindow implements ProcessorSupplier { +class KStreamJoinWindow implements ProcessorSupplier { private final String windowName; @@ -31,11 +32,11 @@ class KStreamJoinWindow implements ProcessorSupplier { } @Override - public Processor get() { + public Processor get() { return new KStreamJoinWindowProcessor(); } - private class KStreamJoinWindowProcessor extends AbstractProcessor { + private class KStreamJoinWindowProcessor extends ContextualProcessor { private WindowStore window; @@ -48,13 +49,13 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K key, final V value) { + public void process(Record record) { // if the key is null, we do not need to put the record into window store // since it will never be considered for join operations - if (key != null) { - context().forward(key, value); + if (record.key() != null) { + context().forward(record); // Every record basically starts a new window. We're using a window store mostly for the retention. - window.put(key, value, context().timestamp()); + window.put(record.key(), record.value(), record.timestamp()); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 53d328f093638..efea46526588d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -16,23 +16,24 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; + import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; +class KStreamKStreamJoin implements ProcessorSupplier { -class KStreamKStreamJoin implements ProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); private final String otherWindowName; @@ -55,61 +56,63 @@ class KStreamKStreamJoin implements ProcessorSupplier { } @Override - public Processor get() { + public Processor get() { return new KStreamKStreamJoinProcessor(); } - private class KStreamKStreamJoinProcessor extends AbstractProcessor { + private class KStreamKStreamJoinProcessor extends ContextualProcessor { private WindowStore otherWindow; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); - otherWindow = (WindowStore) context.getStateStore(otherWindowName); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor( + Thread.currentThread().getName(), context.taskId().toString(), metrics); + otherWindow = context.getStateStore(otherWindowName); } - @Override - public void process(final K key, final V1 value) { + public void process(Record record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record // // we also ignore the record if value is null, because in a key-value data model a null-value indicates // an empty message (ie, there is nothing to be joined) -- this contrast SQL NULL semantics // furthermore, on left/outer joins 'null' in ValueJoiner#apply() indicates a missing record -- // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored - if (key == null || value == null) { - LOG.warn( - "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", - key, value, context().topic(), context().partition(), context().offset() - ); + if (record.key() == null || record.value() == null) { +// LOG.warn( +// "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// key, value, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } boolean needOuterJoin = outer; - final long inputRecordTimestamp = context().timestamp(); + final long inputRecordTimestamp = record.timestamp(); final long timeFrom = Math.max(0L, inputRecordTimestamp - joinBeforeMs); final long timeTo = Math.max(0L, inputRecordTimestamp + joinAfterMs); - try (final WindowStoreIterator iter = otherWindow.fetch(key, timeFrom, timeTo)) { + try (final WindowStoreIterator iter = otherWindow + .fetch(record.key(), timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; final KeyValue otherRecord = iter.next(); context().forward( - key, - joiner.apply(key, value, otherRecord.value), - To.all().withTimestamp(Math.max(inputRecordTimestamp, otherRecord.key))); + record + .withValue( + joiner.apply(record.key(), record.value(), otherRecord.value)) + .withTimestamp(Math.max(inputRecordTimestamp, otherRecord.key))); } if (needOuterJoin) { - context().forward(key, joiner.apply(key, value, null)); + context().forward( + record.withValue(joiner.apply(record.key(), record.value(), null))); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index e2be315e0c142..c2f254d5f433c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -18,16 +18,16 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; -public class KStreamTransformValues implements ProcessorSupplier { +public class KStreamTransformValues implements ProcessorSupplier { private final ValueTransformerWithKeySupplier valueTransformerSupplier; @@ -36,7 +36,7 @@ public class KStreamTransformValues implements ProcessorSupplier } @Override - public Processor get() { + public Processor get() { return new KStreamTransformValuesProcessor<>(valueTransformerSupplier.get()); } @@ -45,7 +45,8 @@ public Set> stores() { return valueTransformerSupplier.stores(); } - public static class KStreamTransformValuesProcessor extends AbstractProcessor { + public static class KStreamTransformValuesProcessor extends + ContextualProcessor { private final ValueTransformerWithKey valueTransformer; @@ -54,14 +55,14 @@ public static class KStreamTransformValuesProcessor extends AbstractPro } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + //TODO valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } @Override - public void process(final K key, final V value) { - context.forward(key, valueTransformer.transform(key, value)); + public void process(Record record) { + context.forward(record.withValue(valueTransformer.transform(record.key(), record.value()))); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 06db4225d2389..b97a87e3b5ed7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -807,13 +807,14 @@ public KGroupedTable groupBy(final KeyValueMapper valueAndTimestampGetterSupplier() { if (processorSupplier instanceof KTableSource) { final KTableSource source = (KTableSource) processorSupplier; // whenever a source ktable is required for getter, it should be materialized source.materialize(); return new KTableSourceValueAndTimestampGetterSupplier<>(source.queryableName()); - } else if (processorSupplier instanceof KStreamAggProcessorSupplier) { + } else if (processorSupplier instanceof KStreamAggregateProcessorSupplier) { return ((AggregationProcessorSupplier) processorSupplier).view(); } else { return ((KTableChangeProcessorSupplier) processorSupplier).view(); @@ -830,9 +831,10 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { } source.enableSendingOldValues(); } else if (processorSupplier instanceof KStreamAggProcessorSupplier) { - ((KStreamAggProcessorSupplier) processorSupplier).enableSendingOldValues(); + ((KStreamAggregateProcessorSupplier) processorSupplier).enableSendingOldValues(); } else { - final KTableProcessorSupplier tableProcessorSupplier = (KTableProcessorSupplier) processorSupplier; + final KTableChangeProcessorSupplier tableProcessorSupplier = + (KTableChangeProcessorSupplier) processorSupplier; if (!tableProcessorSupplier.enableSendingOldValues(forceMaterialization)) { return false; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 6280412d65e7a..c35a0b1babffb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -73,7 +73,7 @@ private class KTableKTableOuterJoinProcessor extends ContextualProcessor> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java index ca2b717f8c9d9..4ef0b55680170 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java @@ -17,23 +17,18 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.internals.CacheFlushListener; class SessionCacheFlushListener implements CacheFlushListener, V> { - private final InternalProcessorContext context; + private final InternalProcessorContext, Change> context; private final ProcessorNode myNode; - SessionCacheFlushListener(final org.apache.kafka.streams.processor.api.ProcessorContext, Change> context) { - this.context = (InternalProcessorContext) context; - myNode = this.context.currentNode(); - } - - SessionCacheFlushListener(final ProcessorContext context) { - this.context = (InternalProcessorContext) context; + SessionCacheFlushListener(final ProcessorContext, Change> context) { + this.context = (InternalProcessorContext, Change>) context; myNode = this.context.currentNode(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java deleted file mode 100644 index 01458c5bf0a00..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionChangeCacheFlushListener.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; -import org.apache.kafka.streams.processor.internals.InternalProcessorContext; -import org.apache.kafka.streams.processor.internals.ProcessorNode; -import org.apache.kafka.streams.state.internals.CacheFlushListener; - -class SessionChangeCacheFlushListener implements CacheFlushListener, V> { - private final InternalProcessorContext context; - private final ProcessorNode myNode; - - SessionChangeCacheFlushListener(final ProcessorContext context) { - this.context = (InternalProcessorContext) context; - myNode = this.context.currentNode(); - } - - @Override - public void apply(final Windowed key, - final V newValue, - final V oldValue, - final long timestamp) { - final ProcessorNode prev = context.currentNode(); - context.setCurrentNode(myNode); - try { - context.forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(key.window().end())); - } finally { - context.setCurrentNode(prev); - } - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListener.java deleted file mode 100644 index 5540376d0cb8c..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListener.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; -import org.apache.kafka.streams.processor.internals.InternalProcessorContext; -import org.apache.kafka.streams.processor.internals.ProcessorNode; -import org.apache.kafka.streams.state.ValueAndTimestamp; -import org.apache.kafka.streams.state.internals.CacheFlushListener; - -import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; - -class TimestampedCacheFlushListener implements CacheFlushListener> { - private final InternalProcessorContext context; - private final ProcessorNode myNode; - - TimestampedCacheFlushListener(final ProcessorContext context) { - this.context = (InternalProcessorContext) context; - myNode = this.context.currentNode(); - } - - @Override - public void apply(final K key, - final ValueAndTimestamp newValue, - final ValueAndTimestamp oldValue, - final long timestamp) { - final ProcessorNode prev = context.currentNode(); - context.setCurrentNode(myNode); - try { - context.forward( - key, - new Change<>(getValueOrNull(newValue), getValueOrNull(oldValue)), - To.all().withTimestamp(newValue != null ? newValue.timestamp() : timestamp)); - } finally { - context.setCurrentNode(prev); - } - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarder.java deleted file mode 100644 index 910dd8fffbf25..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarder.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.To; -import org.apache.kafka.streams.state.internals.WrappedStateStore; - -/** - * This class is used to determine if a processor should forward values to child nodes. - * Forwarding by this class only occurs when caching is not enabled. If caching is enabled, - * forwarding occurs in the flush listener when the cached store flushes. - * - * @param the type of the key - * @param the type of the value - */ -class TimestampedTupleForwarder { - private final ProcessorContext context; - private final boolean sendOldValues; - private final boolean cachingEnabled; - - @SuppressWarnings("unchecked") - TimestampedTupleForwarder(final StateStore store, - final ProcessorContext context, - final TimestampedCacheFlushListener flushListener, - final boolean sendOldValues) { - this.context = context; - this.sendOldValues = sendOldValues; - cachingEnabled = ((WrappedStateStore) store).setFlushListener(flushListener, sendOldValues); - } - - public void maybeForward(final K key, - final V newValue, - final V oldValue) { - if (!cachingEnabled) { - context.forward(key, new Change<>(newValue, sendOldValues ? oldValue : null)); - } - } - - public void maybeForward(final K key, - final V newValue, - final V oldValue, - final long timestamp) { - if (!cachingEnabled) { - context.forward(key, new Change<>(newValue, sendOldValues ? oldValue : null), To.all().withTimestamp(timestamp)); - } - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java index b2f149318fc3a..32b04775c639e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeForwarder.java @@ -16,45 +16,48 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.internals.WrappedStateStore; /** - * This class is used to determine if a processor should forward values to child nodes. - * Forwarding by this class only occurs when caching is not enabled. If caching is enabled, - * forwarding occurs in the flush listener when the cached store flushes. + * This class is used to determine if a processor should forward values to child nodes. Forwarding + * by this class only occurs when caching is not enabled. If caching is enabled, forwarding occurs + * in the flush listener when the cached store flushes. * * @param the type of the key * @param the type of the value */ class TupleChangeForwarder { + private final ProcessorContext> context; private final boolean sendOldValues; private final boolean cachingEnabled; + @SuppressWarnings("unchecked") TupleChangeForwarder(final StateStore store, - final ProcessorContext> context, - final TupleChangeCacheFlushListener flushListener, - final boolean sendOldValues) { + final ProcessorContext> context, + final TupleChangeCacheFlushListener flushListener, + final boolean sendOldValues) { this.context = context; this.sendOldValues = sendOldValues; cachingEnabled = ((WrappedStateStore) store).setFlushListener(flushListener, sendOldValues); } public void maybeForward(final Record record, - final V newValue, - final V oldValue) { + final V newValue, + final V oldValue) { if (!cachingEnabled) { - context.forward(record.withValue(new Change<>(newValue, sendOldValues ? oldValue : null))); + context + .forward(record.withValue(new Change<>(newValue, sendOldValues ? oldValue : null))); } } public void maybeForward(final Record record, - final V newValue, - final V oldValue, - final long timestamp) { + final V newValue, + final V oldValue, + final long timestamp) { if (!cachingEnabled) { context.forward(record .withValue(new Change<>(newValue, sendOldValues ? oldValue : null)) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java index edefd1f4dcdd9..6b735f5b56a01 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/CombinedKeySchema.java @@ -51,15 +51,6 @@ public CombinedKeySchema(final Supplier foreignKeySerdeTopicSupplier, } @SuppressWarnings("unchecked") - public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { - primaryKeySerdeTopic = undecoratedPrimaryKeySerdeTopicSupplier.get(); - foreignKeySerdeTopic = undecoratedForeignKeySerdeTopicSupplier.get(); - primaryKeySerializer = primaryKeySerializer == null ? (Serializer) context.keySerde().serializer() : primaryKeySerializer; - primaryKeyDeserializer = primaryKeyDeserializer == null ? (Deserializer) context.keySerde().deserializer() : primaryKeyDeserializer; - foreignKeySerializer = foreignKeySerializer == null ? (Serializer) context.keySerde().serializer() : foreignKeySerializer; - foreignKeyDeserializer = foreignKeyDeserializer == null ? (Deserializer) context.keySerde().deserializer() : foreignKeyDeserializer; - } - public void init(final ProcessorContext context) { primaryKeySerdeTopic = undecoratedPrimaryKeySerdeTopicSupplier.get(); foreignKeySerdeTopic = undecoratedForeignKeySerdeTopicSupplier.get(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java index 97878750dc1f6..f33d482a46527 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java @@ -21,10 +21,11 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.internals.Murmur3; @@ -40,7 +41,7 @@ import static org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionWrapper.Instruction.PROPAGATE_NULL_IF_NO_FK_VAL_AVAILABLE; import static org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionWrapper.Instruction.PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE; -public class ForeignJoinSubscriptionSendProcessorSupplier implements ProcessorSupplier> { +public class ForeignJoinSubscriptionSendProcessorSupplier implements ProcessorSupplier, KO, SubscriptionWrapper> { private static final Logger LOG = LoggerFactory.getLogger(ForeignJoinSubscriptionSendProcessorSupplier.class); private final Function foreignKeyExtractor; @@ -65,11 +66,11 @@ public ForeignJoinSubscriptionSendProcessorSupplier(final Function foreig } @Override - public Processor> get() { + public Processor, KO, SubscriptionWrapper> get() { return new UnbindChangeProcessor(); } - private class UnbindChangeProcessor extends AbstractProcessor> { + private class UnbindChangeProcessor extends ContextualProcessor, KO, SubscriptionWrapper> { private Sensor droppedRecordsSensor; private String foreignKeySerdeTopic; @@ -77,7 +78,7 @@ private class UnbindChangeProcessor extends AbstractProcessor> { @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext> context) { super.init(context); foreignKeySerdeTopic = foreignKeySerdeTopicSupplier.get(); valueSerdeTopic = valueSerdeTopicSupplier.get(); @@ -96,28 +97,28 @@ public void init(final ProcessorContext context) { } @Override - public void process(final K key, final Change change) { - final long[] currentHash = change.newValue == null ? + public void process(Record> record) { + final long[] currentHash = record.value().newValue == null ? null : - Murmur3.hash128(valueSerializer.serialize(valueSerdeTopic, change.newValue)); + Murmur3.hash128(valueSerializer.serialize(valueSerdeTopic, record.value().newValue)); - if (change.oldValue != null) { - final KO oldForeignKey = foreignKeyExtractor.apply(change.oldValue); + if (record.value().oldValue != null) { + final KO oldForeignKey = foreignKeyExtractor.apply(record.value().oldValue); if (oldForeignKey == null) { - LOG.warn( - "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - change.oldValue, context().topic(), context().partition(), context().offset() - ); +// LOG.warn( +// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// change.oldValue, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } - if (change.newValue != null) { - final KO newForeignKey = foreignKeyExtractor.apply(change.newValue); + if (record.value().newValue != null) { + final KO newForeignKey = foreignKeyExtractor.apply(record.value().newValue); if (newForeignKey == null) { - LOG.warn( - "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - change.newValue, context().topic(), context().partition(), context().offset() - ); +// LOG.warn( +// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// change.newValue, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); return; } @@ -129,17 +130,17 @@ change.newValue, context().topic(), context().partition(), context().offset() if (!Arrays.equals(serialNewForeignKey, serialOldForeignKey)) { //Different Foreign Key - delete the old key value and propagate the new one. //Delete it from the oldKey's state store - context().forward(oldForeignKey, new SubscriptionWrapper<>(currentHash, DELETE_KEY_NO_PROPAGATE, key)); + context().forward(record.withKey(oldForeignKey).withValue(new SubscriptionWrapper<>(currentHash, DELETE_KEY_NO_PROPAGATE, record.key()))); //Add to the newKey's state store. Additionally, propagate null if no FK is found there, //since we must "unset" any output set by the previous FK-join. This is true for both INNER //and LEFT join. } - context().forward(newForeignKey, new SubscriptionWrapper<>(currentHash, PROPAGATE_NULL_IF_NO_FK_VAL_AVAILABLE, key)); + context().forward(record.withKey(newForeignKey).withValue(new SubscriptionWrapper<>(currentHash, PROPAGATE_NULL_IF_NO_FK_VAL_AVAILABLE, record.key()))); } else { //A simple propagatable delete. Delete from the state store and propagate the delete onwards. - context().forward(oldForeignKey, new SubscriptionWrapper<>(currentHash, DELETE_KEY_AND_PROPAGATE, key)); + context().forward(record.withKey(oldForeignKey).withValue(new SubscriptionWrapper<>(currentHash, DELETE_KEY_AND_PROPAGATE, record.key()))); } - } else if (change.newValue != null) { + } else if (record.value().newValue != null) { //change.oldValue is null, which means it was deleted at least once before, or it is brand new. //In either case, we only need to propagate if the FK_VAL is available, as the null from the delete would //have been propagated otherwise. @@ -151,15 +152,15 @@ change.newValue, context().topic(), context().partition(), context().offset() } else { instruction = PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE; } - final KO newForeignKey = foreignKeyExtractor.apply(change.newValue); + final KO newForeignKey = foreignKeyExtractor.apply(record.value().newValue); if (newForeignKey == null) { - LOG.warn( - "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", - change.newValue, context().topic(), context().partition(), context().offset() - ); +// LOG.warn( +// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", +// change.newValue, context().topic(), context().partition(), context().offset() +// ); droppedRecordsSensor.record(); } else { - context().forward(newForeignKey, new SubscriptionWrapper<>(currentHash, instruction, key)); + context().forward(record.withKey(newForeignKey).withValue(new SubscriptionWrapper<>(currentHash, instruction, record.key()))); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java index 017087e6404c8..916a8644459ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java @@ -23,7 +23,7 @@ import org.apache.kafka.streams.kstream.internals.KStreamSessionWindowAggregate; import org.apache.kafka.streams.kstream.internals.KStreamSlidingWindowAggregate; import org.apache.kafka.streams.kstream.internals.KStreamWindowAggregate; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; public final class GraphGraceSearchUtil { private GraphGraceSearchUtil() {} @@ -72,7 +72,7 @@ private static long findAndVerifyWindowGrace(final GraphNode graphNode, final St private static Long extractGracePeriod(final GraphNode node) { if (node instanceof StatefulProcessorNode) { - final ProcessorSupplier processorSupplier = ((StatefulProcessorNode) node).processorParameters().oldProcessorSupplier(); + final ProcessorSupplier processorSupplier = ((StatefulProcessorNode) node).processorParameters().processorSupplier(); if (processorSupplier instanceof KStreamWindowAggregate) { final KStreamWindowAggregate kStreamWindowAggregate = (KStreamWindowAggregate) processorSupplier; final Windows windows = kStreamWindowAggregate.windows(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java index 8860a8f44ada7..6d54cefe38fe6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java @@ -21,33 +21,21 @@ import org.apache.kafka.streams.kstream.internals.KTableKTableJoinMerger; import org.apache.kafka.streams.kstream.internals.KTableSource; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ProcessorAdapter; /** - * Class used to represent a {@link ProcessorSupplier} and the name - * used to register it with the {@link org.apache.kafka.streams.processor.internals.InternalTopologyBuilder} - * - * Used by the Join nodes as there are several parameters, this abstraction helps - * keep the number of arguments more reasonable. + * Class used to represent a {@link ProcessorSupplier} and the name used to register it with the + * {@link org.apache.kafka.streams.processor.internals.InternalTopologyBuilder} + *

+ * Used by the Join nodes as there are several parameters, this abstraction helps keep the number of + * arguments more reasonable. */ public class ProcessorParameters { - // During the transition to KIP-478, we capture arguments passed from the old API to simplify - // the performance of casts that we still need to perform. This will eventually be removed. - private final org.apache.kafka.streams.processor.ProcessorSupplier oldProcessorSupplier; private final ProcessorSupplier processorSupplier; private final String processorName; - public ProcessorParameters(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, - final String processorName) { - oldProcessorSupplier = processorSupplier; - this.processorSupplier = () -> ProcessorAdapter.adapt(processorSupplier.get()); - this.processorName = processorName; - } - public ProcessorParameters(final ProcessorSupplier processorSupplier, final String processorName) { - oldProcessorSupplier = null; this.processorSupplier = processorSupplier; this.processorName = processorName; } @@ -56,20 +44,17 @@ public ProcessorSupplier processorSupplier() { return processorSupplier; } - public org.apache.kafka.streams.processor.ProcessorSupplier oldProcessorSupplier() { - return oldProcessorSupplier; - } - @SuppressWarnings("unchecked") KTableSource kTableSourceSupplier() { // This cast always works because KTableSource hasn't been converted yet. - return oldProcessorSupplier == null + return processorSupplier == null ? null - : !(oldProcessorSupplier instanceof KTableSource) - ? null - : (KTableSource) oldProcessorSupplier; + : !(processorSupplier instanceof KTableSource) + ? null + : (KTableSource) processorSupplier; } + @SuppressWarnings("unchecked") KTableChangeProcessorSupplier kTableChangeProcessorSupplier() { // This cast always works because KTableProcessorSupplier hasn't been converted yet. return (KTableChangeProcessorSupplier) processorSupplier; @@ -77,7 +62,7 @@ KTableChangeProcessorSupplier kTableChangeProcessorS @SuppressWarnings("unchecked") KTableKTableJoinMerger kTableKTableJoinMergerProcessorSupplier() { - return (KTableKTableJoinMerger) oldProcessorSupplier; + return (KTableKTableJoinMerger) processorSupplier; } public String processorName() { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java index e3d22fa37562f..3d6a9b15bb196 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java @@ -101,8 +101,8 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { } // temporary hack until KIP-478 is fully implemented - final org.apache.kafka.streams.processor.ProcessorSupplier oldProcessorSupplier = - processorParameters().oldProcessorSupplier(); + final ProcessorSupplier oldProcessorSupplier = + processorParameters().processorSupplier(); if (oldProcessorSupplier != null && oldProcessorSupplier.stores() != null) { for (final StoreBuilder storeBuilder : oldProcessorSupplier.stores()) { topologyBuilder.addStateStore(storeBuilder, processorName); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java index 3b0a572c1141b..edb1443c5c1bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java @@ -57,10 +57,10 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { new TimestampedKeyValueStoreMaterializer<>((MaterializedInternal>) materializedInternal).materialize(); final String processorName = processorParameters.processorName(); - final KTableSource ktableSource = processorParameters.kTableSourceSupplier(); + final KTableSource kTableSource = processorParameters.kTableSourceSupplier(); topologyBuilder.addProcessor(processorName, processorParameters.processorSupplier(), parentNodeNames()); - if (storeBuilder != null && ktableSource.materialized()) { + if (storeBuilder != null && kTableSource.materialized()) { topologyBuilder.addStateStore(storeBuilder, processorName); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index 6b1378b44bc83..e5f591c9f3ef8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -69,7 +69,7 @@ public Map initialize() { final Map storeNameToTopic = topology.storeToChangelogTopic(); for (final String storeName : storeNames) { final String sourceTopic = storeNameToTopic.get(storeName); - final SourceNode source = topology.source(sourceTopic); + final SourceNode source = topology.source(sourceTopic); deserializers.put( sourceTopic, new RecordDeserializer( @@ -111,7 +111,7 @@ public void update(final ConsumerRecord record) { processorContext.timestamp(), processorContext.headers() ); - ((SourceNode) sourceNodeAndDeserializer.sourceNode()).process(toProcess); + ((SourceNode) sourceNodeAndDeserializer.sourceNode()).process(toProcess); } offsets.put(new TopicPartition(record.topic(), record.partition()), record.offset() + 1); @@ -138,6 +138,7 @@ public void close(final boolean wipeStateStore) throws IOException { } } + @SuppressWarnings("unchecked") private void initTopology() { for (final ProcessorNode node : this.topology.processors()) { processorContext.setCurrentNode(node); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index bb18f92ef23d2..c8b77a23a2946 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -229,7 +229,7 @@ Processor describe() { // even if it can be matched by multiple regex patterns. Only used by SourceNodeFactory private final Map topicToPatterns = new HashMap<>(); - private class SourceNodeFactory extends NodeFactory { + private class SourceNodeFactory extends NodeFactory { private final List topics; private final Pattern pattern; private final Deserializer keyDeserializer; @@ -277,7 +277,7 @@ List getTopics(final Collection subscribedTopics) { } @Override - public ProcessorNode build() { + public ProcessorNode build() { return new SourceNode<>(name, timestampExtractor, keyDeserializer, valDeserializer); } @@ -733,14 +733,14 @@ private void connectProcessorAndStateStore(final String processorName, } } - private Set> findSourcesForProcessorPredecessors(final String[] predecessors) { - final Set> sourceNodes = new HashSet<>(); + private Set> findSourcesForProcessorPredecessors(final String[] predecessors) { + final Set> sourceNodes = new HashSet<>(); for (final String predecessor : predecessors) { final NodeFactory nodeFactory = nodeFactories.get(predecessor); if (nodeFactory instanceof SourceNodeFactory) { - sourceNodes.add((SourceNodeFactory) nodeFactory); + sourceNodes.add((SourceNodeFactory) nodeFactory); } else if (nodeFactory instanceof ProcessorNodeFactory) { - sourceNodes.addAll(findSourcesForProcessorPredecessors(((ProcessorNodeFactory) nodeFactory).predecessors)); + sourceNodes.addAll(findSourcesForProcessorPredecessors((nodeFactory).predecessors)); } } return sourceNodes; @@ -759,10 +759,10 @@ private void connectStateStoreNameToSourceTopicsOrPattern final Set sourceTopics = new HashSet<>(); final Set sourcePatterns = new HashSet<>(); - final Set> sourceNodesForPredecessor = + final Set> sourceNodesForPredecessor = findSourcesForProcessorPredecessors(processorNodeFactory.predecessors); - for (final SourceNodeFactory sourceNodeFactory : sourceNodesForPredecessor) { + for (final SourceNodeFactory sourceNodeFactory : sourceNodesForPredecessor) { if (sourceNodeFactory.pattern != null) { sourcePatterns.add(sourceNodeFactory.pattern); } else { @@ -897,7 +897,7 @@ private ProcessorTopology build(final Set nodeGroup) { Objects.requireNonNull(applicationId, "topology has not completed optimization"); final Map> processorMap = new LinkedHashMap<>(); - final Map> topicSourceMap = new HashMap<>(); + final Map> topicSourceMap = new HashMap<>(); final Map> topicSinkMap = new HashMap<>(); final Map stateStoreMap = new LinkedHashMap<>(); final Set repartitionTopics = new HashSet<>(); @@ -918,8 +918,8 @@ private ProcessorTopology build(final Set nodeGroup) { } else if (factory instanceof SourceNodeFactory) { buildSourceNode(topicSourceMap, repartitionTopics, - (SourceNodeFactory) factory, - (SourceNode) node); + (SourceNodeFactory) factory, + (SourceNode) node); } else if (factory instanceof SinkNodeFactory) { buildSinkNode(processorMap, @@ -974,10 +974,10 @@ private static ProcessorNode getPro return (ProcessorNode) processorMap.get(predecessor); } - private void buildSourceNode(final Map> topicSourceMap, + private void buildSourceNode(final Map> topicSourceMap, final Set repartitionTopics, - final SourceNodeFactory sourceNodeFactory, - final SourceNode node) { + final SourceNodeFactory sourceNodeFactory, + final SourceNode node) { final List topics = (sourceNodeFactory.pattern != null) ? sourceNodeFactory.getTopics(subscriptionUpdates()) : @@ -1140,7 +1140,7 @@ private RepartitionTopicConfig buildRepartitionTopicConfig(final String internal private void setRegexMatchedTopicsToSourceNodes() { if (hasSubscriptionUpdates()) { for (final String nodeName : nodeToSourcePatterns.keySet()) { - final SourceNodeFactory sourceNode = (SourceNodeFactory) nodeFactories.get(nodeName); + final SourceNodeFactory sourceNode = (SourceNodeFactory) nodeFactories.get(nodeName); final List sourceTopics = sourceNode.getTopics(subscriptionUpdates); //need to update nodeToSourceTopics and sourceTopicNames with topics matched from given regex nodeToSourceTopics.put(nodeName, sourceTopics); @@ -1330,7 +1330,7 @@ private boolean isGlobalSource(final String nodeName) { final NodeFactory nodeFactory = nodeFactories.get(nodeName); if (nodeFactory instanceof SourceNodeFactory) { - final List topics = ((SourceNodeFactory) nodeFactory).topics; + final List topics = ((SourceNodeFactory) nodeFactory).topics; return topics != null && topics.size() == 1 && globalTopics.contains(topics.get(0)); } return false; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index a8c32c7882c9d..3943e25472f4c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -46,7 +45,7 @@ public class ProcessorNode { public final Set stateStores; - private InternalProcessorContext internalProcessorContext; + private InternalProcessorContext internalProcessorContext; private String threadId; private Sensor processSensor; @@ -105,8 +104,7 @@ public void addChild(final ProcessorNode child) { childByName.put(child.name, child); } - @SuppressWarnings("unchecked") - public void init(final InternalProcessorContext context) { + public void init(final InternalProcessorContext context) { if (!closed) throw new IllegalStateException("The processor is not closed"); @@ -116,7 +114,7 @@ public void init(final InternalProcessorContext context) { maybeMeasureLatency( () -> { if (processor != null) { - processor.init((ProcessorContext) context); + processor.init(context); } }, time, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java index 0a0118ae0087a..8c6811de47fd6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java @@ -33,8 +33,8 @@ public class ProcessorTopology { private final Logger log = LoggerFactory.getLogger(ProcessorTopology.class); private final List> processorNodes; - private final Map> sourceNodesByName; - private final Map> sourceNodesByTopic; + private final Map> sourceNodesByName; + private final Map> sourceNodesByTopic; private final Map> sinksByTopic; private final Set terminalNodes; private final List stateStores; @@ -46,7 +46,7 @@ public class ProcessorTopology { private final Map storeToChangelogTopic; public ProcessorTopology(final List> processorNodes, - final Map> sourceNodesByTopic, + final Map> sourceNodesByTopic, final Map> sinksByTopic, final List stateStores, final List globalStateStores, @@ -69,7 +69,7 @@ public ProcessorTopology(final List> processorNodes, } this.sourceNodesByName = new HashMap<>(); - for (final SourceNode source : sourceNodesByTopic.values()) { + for (final SourceNode source : sourceNodesByTopic.values()) { sourceNodesByName.put(source.name(), source); } } @@ -78,11 +78,11 @@ public Set sourceTopics() { return sourceNodesByTopic.keySet(); } - public SourceNode source(final String topic) { + public SourceNode source(final String topic) { return sourceNodesByTopic.get(topic); } - public Set> sources() { + public Set> sources() { return new HashSet<>(sourceNodesByTopic.values()); } @@ -151,9 +151,9 @@ public boolean hasPersistentGlobalStore() { public void updateSourceTopics(final Map> allSourceTopicsByNodeName) { sourceNodesByTopic.clear(); - for (final Map.Entry> sourceNodeEntry : sourceNodesByName.entrySet()) { + for (final Map.Entry> sourceNodeEntry : sourceNodesByName.entrySet()) { final String sourceNodeName = sourceNodeEntry.getKey(); - final SourceNode sourceNode = sourceNodeEntry.getValue(); + final SourceNode sourceNode = sourceNodeEntry.getValue(); final List updatedSourceTopics = allSourceTopicsByNodeName.get(sourceNodeName); if (updatedSourceTopics == null) { @@ -211,10 +211,10 @@ public String toString() { * @return A string representation of this instance. */ public String toString(final String indent) { - final Map, List> sourceToTopics = new HashMap<>(); - for (final Map.Entry> sourceNodeEntry : sourceNodesByTopic.entrySet()) { + final Map, List> sourceToTopics = new HashMap<>(); + for (final Map.Entry> sourceNodeEntry : sourceNodesByTopic.entrySet()) { final String topic = sourceNodeEntry.getKey(); - final SourceNode source = sourceNodeEntry.getValue(); + final SourceNode source = sourceNodeEntry.getValue(); sourceToTopics.computeIfAbsent(source, s -> new ArrayList<>()); sourceToTopics.get(source).add(topic); } @@ -222,8 +222,8 @@ public String toString(final String indent) { final StringBuilder sb = new StringBuilder(indent + "ProcessorTopology:\n"); // start from sources - for (final Map.Entry, List> sourceNodeEntry : sourceToTopics.entrySet()) { - final SourceNode source = sourceNodeEntry.getKey(); + for (final Map.Entry, List> sourceNodeEntry : sourceToTopics.entrySet()) { + final SourceNode source = sourceNodeEntry.getKey(); final List topics = sourceNodeEntry.getValue(); sb.append(source.toString(indent + "\t")) .append(topicsToString(indent + "\t", topics)) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 269dff2822290..86b1b442be6c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -29,11 +29,11 @@ class RecordDeserializer { private final Logger log; - private final SourceNode sourceNode; + private final SourceNode sourceNode; private final Sensor droppedRecordsSensor; private final DeserializationExceptionHandler deserializationExceptionHandler; - RecordDeserializer(final SourceNode sourceNode, + RecordDeserializer(final SourceNode sourceNode, final DeserializationExceptionHandler deserializationExceptionHandler, final LogContext logContext, final Sensor droppedRecordsSensor) { @@ -98,7 +98,7 @@ ConsumerRecord deserialize(final ProcessorContext processorConte } } - SourceNode sourceNode() { + SourceNode sourceNode() { return sourceNode; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index df7e834f092b1..6f0db8a648865 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -39,7 +39,7 @@ public class RecordQueue { public static final long UNKNOWN = ConsumerRecord.NO_TIMESTAMP; private final Logger log; - private final SourceNode source; + private final SourceNode source; private final TopicPartition partition; private final ProcessorContext processorContext; private final TimestampExtractor timestampExtractor; @@ -52,7 +52,7 @@ public class RecordQueue { private final Sensor droppedRecordsSensor; RecordQueue(final TopicPartition partition, - final SourceNode source, + final SourceNode source, final TimestampExtractor timestampExtractor, final DeserializationExceptionHandler deserializationExceptionHandler, final InternalProcessorContext processorContext, @@ -85,7 +85,7 @@ void setPartitionTime(final long partitionTime) { * * @return SourceNode */ - public SourceNode source() { + public SourceNode source() { return source; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 2efa537ea4ffc..5d571a422be44 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -31,7 +31,7 @@ public class SinkNode extends ProcessorNode topicExtractor; private final StreamPartitioner partitioner; - private InternalProcessorContext context; + private InternalProcessorContext context; SinkNode(final String name, final TopicNameExtractor topicExtractor, @@ -55,7 +55,7 @@ public void addChild(final ProcessorNode child) { } @Override - public void init(final InternalProcessorContext context) { + public void init(final InternalProcessorContext context) { super.init(context); this.context = context; final Serializer contextKeySerializer = ProcessorContextUtils.getKeySerializer(context); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 7198f2f17e93e..9ff3473b3c59c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -26,9 +26,9 @@ import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeyDeserializer; import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareValueDeserializer; -public class SourceNode extends ProcessorNode { +public class SourceNode extends ProcessorNode { - private InternalProcessorContext context; + private InternalProcessorContext context; private Deserializer keyDeserializer; private Deserializer valDeserializer; private final TimestampExtractor timestampExtractor; @@ -59,7 +59,7 @@ VIn deserializeValue(final String topic, final Headers headers, final byte[] dat } @Override - public void init(final InternalProcessorContext context) { + public void init(final InternalProcessorContext context) { // It is important to first create the sensor before calling init on the // parent object. Otherwise due to backwards compatibility an empty sensor // without parent is created with the same name. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index e2c5e03cb15ef..c4400d0f05faf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -312,6 +312,7 @@ public void suspend() { } } + @SuppressWarnings("unchecked") private void closeTopology() { log.trace("Closing processor topology"); @@ -793,6 +794,7 @@ private String getStacktraceString(final RuntimeException e) { * @throws IllegalStateException if the current node is not null * @throws TaskMigratedException if the task producer got fenced (EOS only) */ + @SuppressWarnings("unchecked") @Override public void punctuate(final ProcessorNode node, final long timestamp, @@ -828,6 +830,7 @@ public void punctuate(final ProcessorNode node, } } + @SuppressWarnings("unchecked") private void updateProcessorContext(final ProcessorNode currNode, final long wallClockTime, final ProcessorRecordContext recordContext) { @@ -921,6 +924,7 @@ public Map purgeableOffsets() { return purgeableConsumedOffsets; } + @SuppressWarnings("unchecked") private void initializeTopology() { // initialize the task by initializing all its processor nodes in the topology log.trace("Initializing processor nodes of the topology"); @@ -1095,6 +1099,7 @@ long decodeTimestamp(final String encryptedString) { } } + @SuppressWarnings("unchecked") public InternalProcessorContext processorContext() { return processorContext; } @@ -1216,7 +1221,7 @@ private RecordQueueCreator(final LogContext logContext, } public RecordQueue createQueue(final TopicPartition partition) { - final SourceNode source = topology.source(partition.topic()); + final SourceNode source = topology.source(partition.topic()); if (source == null) { throw new TopologyException( "Topic is unknown to the topology. " + diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 1ca13ce6d7669..ce58724048405 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -1526,9 +1526,9 @@ public void shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningT final ProcessorTopology topology = TopologyWrapper.getInternalTopologyBuilder(builder.build()).setApplicationId("X").buildTopology(); - final SourceNode originalSourceNode = topology.source("topic-1"); + final SourceNode originalSourceNode = topology.source("topic-1"); - for (final SourceNode sourceNode : topology.sources()) { + for (final SourceNode sourceNode : topology.sources()) { if (sourceNode.name().equals(originalSourceNode.name())) { assertNull(sourceNode.getTimestampExtractor()); } else { @@ -1555,9 +1555,9 @@ public void shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningT final ProcessorTopology topology = TopologyWrapper.getInternalTopologyBuilder(builder.build()).setApplicationId("X").buildTopology(); - final SourceNode originalSourceNode = topology.source("topic-1"); + final SourceNode originalSourceNode = topology.source("topic-1"); - for (final SourceNode sourceNode : topology.sources()) { + for (final SourceNode sourceNode : topology.sources()) { if (sourceNode.name().equals(originalSourceNode.name())) { assertNull(sourceNode.getTimestampExtractor()); } else { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java index 38ef5c68401a9..3a2d1e7942c9b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java @@ -31,7 +31,7 @@ public class TimestampedCacheFlushListenerTest { @Test public void shouldForwardValueTimestampIfNewValueExists() { - final InternalProcessorContext context = mock(InternalProcessorContext.class); + final InternalProcessorContext> context = mock(InternalProcessorContext.class); expect(context.currentNode()).andReturn(null).anyTimes(); context.setCurrentNode(null); context.setCurrentNode(null); @@ -42,7 +42,7 @@ public void shouldForwardValueTimestampIfNewValueExists() { expectLastCall(); replay(context); - new TimestampedCacheFlushListener<>(context).apply( + new TupleChangeCacheFlushListener<>(context).apply( "key", ValueAndTimestamp.make("newValue", 42L), ValueAndTimestamp.make("oldValue", 21L), @@ -53,7 +53,7 @@ public void shouldForwardValueTimestampIfNewValueExists() { @Test public void shouldForwardParameterTimestampIfNewValueIsNull() { - final InternalProcessorContext context = mock(InternalProcessorContext.class); + final InternalProcessorContext> context = mock(InternalProcessorContext.class); expect(context.currentNode()).andReturn(null).anyTimes(); context.setCurrentNode(null); context.setCurrentNode(null); @@ -64,7 +64,7 @@ public void shouldForwardParameterTimestampIfNewValueIsNull() { expectLastCall(); replay(context); - new TimestampedCacheFlushListener<>(context).apply( + new TupleChangeCacheFlushListener<>(context).apply( "key", null, ValueAndTimestamp.make("oldValue", 21L), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarderTest.java index 52a5fcf24d4ce..24a5e42b3fb28 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedTupleForwarderTest.java @@ -16,9 +16,9 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.junit.Test; @@ -39,12 +39,12 @@ public void shouldSetFlushListenerOnWrappedStateStore() { private void setFlushListener(final boolean sendOldValues) { final WrappedStateStore> store = mock(WrappedStateStore.class); - final TimestampedCacheFlushListener flushListener = mock(TimestampedCacheFlushListener.class); + final TupleChangeCacheFlushListener flushListener = mock(TupleChangeCacheFlushListener.class); expect(store.setFlushListener(flushListener, sendOldValues)).andReturn(false); replay(store); - new TimestampedTupleForwarder<>(store, null, flushListener, sendOldValues); + new TupleChangeForwarder<>(store, null, flushListener, sendOldValues); verify(store); } @@ -57,23 +57,29 @@ public void shouldForwardRecordsIfWrappedStateStoreDoesNotCache() { private void shouldForwardRecordsIfWrappedStateStoreDoesNotCache(final boolean sendOldValues) { final WrappedStateStore store = mock(WrappedStateStore.class); - final ProcessorContext context = mock(ProcessorContext.class); + final ProcessorContext> context = mock(ProcessorContext.class); expect(store.setFlushListener(null, sendOldValues)).andReturn(false); if (sendOldValues) { - context.forward("key1", new Change<>("newValue1", "oldValue1")); - context.forward("key2", new Change<>("newValue2", "oldValue2"), To.all().withTimestamp(42L)); + final Record> record1 = new Record<>("key1", new Change<>("newValue1", "oldValue1"), 0L); + context.forward(record1); + final Record> record2 = new Record<>("key2", new Change<>("newValue2", "oldValue2"), 42L); + context.forward(record2); } else { - context.forward("key1", new Change<>("newValue1", null)); - context.forward("key2", new Change<>("newValue2", null), To.all().withTimestamp(42L)); + final Record> record1 = new Record<>("key1", new Change<>("newValue1", null), 0L); + context.forward(record1); + final Record> record2 = new Record<>("key2", new Change<>("newValue2", null), 42L); + context.forward(record2); } expectLastCall(); replay(store, context); - final TimestampedTupleForwarder forwarder = - new TimestampedTupleForwarder<>(store, context, null, sendOldValues); - forwarder.maybeForward("key1", "newValue1", "oldValue1"); - forwarder.maybeForward("key2", "newValue2", "oldValue2", 42L); + final TupleChangeForwarder forwarder = + new TupleChangeForwarder<>(store, context, null, sendOldValues); + final Record record1 = new Record<>("key1", "newValue1", 0L); + forwarder.maybeForward(record1, "newValue1", "oldValue1"); + final Record record2 = new Record<>("key2", "newValue2", 0L); + forwarder.maybeForward(record2, "newValue2", "oldValue2", 42L); verify(store, context); } @@ -81,15 +87,17 @@ private void shouldForwardRecordsIfWrappedStateStoreDoesNotCache(final boolean s @Test public void shouldNotForwardRecordsIfWrappedStateStoreDoesCache() { final WrappedStateStore store = mock(WrappedStateStore.class); - final ProcessorContext context = mock(ProcessorContext.class); + final ProcessorContext> context = mock(ProcessorContext.class); expect(store.setFlushListener(null, false)).andReturn(true); replay(store, context); - final TimestampedTupleForwarder forwarder = - new TimestampedTupleForwarder<>(store, context, null, false); - forwarder.maybeForward("key", "newValue", "oldValue"); - forwarder.maybeForward("key", "newValue", "oldValue", 42L); + final TupleChangeForwarder forwarder = + new TupleChangeForwarder<>(store, context, null, false); + final Record> record1 = new Record<>("key", null, 0L); + forwarder.maybeForward(record1, "newValue", "oldValue"); + final Record> record2 = new Record<>("key", null, 0L); + forwarder.maybeForward(record2, "newValue", "oldValue", 42L); verify(store, context); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java index 3cc06be7858f8..8b36b7fb5e13a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java @@ -59,10 +59,10 @@ public class GlobalStateTaskTest { private final String topic2 = "t2"; private final TopicPartition t1 = new TopicPartition(topic1, 1); private final TopicPartition t2 = new TopicPartition(topic2, 1); - private final MockSourceNode sourceOne = new MockSourceNode<>( + private final MockSourceNode sourceOne = new MockSourceNode<>( new StringDeserializer(), new StringDeserializer()); - private final MockSourceNode sourceTwo = new MockSourceNode<>( + private final MockSourceNode sourceTwo = new MockSourceNode<>( new IntegerDeserializer(), new IntegerDeserializer()); private final MockProcessorNode processorOne = new MockProcessorNode<>(); @@ -79,7 +79,7 @@ public class GlobalStateTaskTest { @Before public void before() { final Set storeNames = Utils.mkSet("t1-store", "t2-store"); - final Map> sourceByTopics = new HashMap<>(); + final Map> sourceByTopics = new HashMap<>(); sourceByTopics.put(topic1, sourceOne); sourceByTopics.put(topic2, sourceTwo); final Map storeToTopic = new HashMap<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyFactories.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyFactories.java index b4cef8ad445bf..57e4490d73bd6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyFactories.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyFactories.java @@ -27,7 +27,7 @@ private ProcessorTopologyFactories() {} public static ProcessorTopology with(final List> processorNodes, - final Map> sourcesByTopic, + final Map> sourcesByTopic, final List stateStoresByName, final Map storeToChangelogTopic) { return new ProcessorTopology(processorNodes, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index 4ae9d32bc44fd..7299067dd3376 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -68,7 +68,7 @@ public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { assertEquals(rawRecord.headers(), record.headers()); } - static class TheSourceNode extends SourceNode { + static class TheSourceNode extends SourceNode { private final boolean keyThrowsException; private final boolean valueThrowsException; private final Object key; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java index 895e0452e1c31..6929bb8e4c116 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -64,7 +64,7 @@ public class RecordQueueTest { StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class), new MockRecordCollector() ); - private final MockSourceNode mockSourceNodeWithMetrics + private final MockSourceNode mockSourceNodeWithMetrics = new MockSourceNode<>(intDeserializer, intDeserializer); private final RecordQueue queue = new RecordQueue( new TopicPartition("topic", 1), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 3a6a7c55872b1..7192ab6a1f84f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -135,9 +135,9 @@ public class StreamTaskTest { private final Serializer intSerializer = Serdes.Integer().serializer(); private final Deserializer intDeserializer = Serdes.Integer().deserializer(); - private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); - private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); - private final MockSourceNode source3 = new MockSourceNode(intDeserializer, intDeserializer) { + private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source3 = new MockSourceNode(intDeserializer, intDeserializer) { @Override public void process(final Record record) { throw new RuntimeException("KABOOM!"); @@ -148,7 +148,7 @@ public void close() { throw new RuntimeException("KABOOM!"); } }; - private final MockSourceNode timeoutSource = new MockSourceNode(intDeserializer, intDeserializer) { + private final MockSourceNode timeoutSource = new MockSourceNode(intDeserializer, intDeserializer) { @Override public void process(final Record record) { throw new TimeoutException("Kaboom!"); @@ -190,7 +190,7 @@ public void punctuate(final long timestamp) { }; private static ProcessorTopology withRepartitionTopics(final List> processorNodes, - final Map> sourcesByTopic, + final Map> sourcesByTopic, final Set repartitionTopics) { return new ProcessorTopology(processorNodes, sourcesByTopic, @@ -202,7 +202,7 @@ private static ProcessorTopology withRepartitionTopics(final List> processorNodes, - final Map> sourcesByTopic) { + final Map> sourcesByTopic) { return new ProcessorTopology(processorNodes, sourcesByTopic, emptyMap(), @@ -619,7 +619,7 @@ public void shouldRecordE2ELatencyOnSourceNodeAndTerminalNodes() { metrics = new Metrics(new MetricConfig().recordLevel(Sensor.RecordingLevel.INFO), time); // Create a processor that only forwards even keys to test the metrics at the source and terminal nodes - final MockSourceNode evenKeyForwardingSourceNode = new MockSourceNode(intDeserializer, intDeserializer) { + final MockSourceNode evenKeyForwardingSourceNode = new MockSourceNode(intDeserializer, intDeserializer) { InternalProcessorContext context; @Override @@ -2457,7 +2457,7 @@ private StreamTask createStatelessTask(final StreamsConfig config, ); } - private StreamTask createStatelessTaskWithForwardingTopology(final SourceNode sourceNode) { + private StreamTask createStatelessTaskWithForwardingTopology(final SourceNode sourceNode) { final ProcessorTopology topology = withSources( asList(sourceNode, processorStreamTime), singletonMap(topic1, sourceNode) diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java index 370dca75ce774..c32c136d4f451 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java @@ -40,7 +40,7 @@ import java.util.Properties; import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener; -public class MockInternalProcessorContext extends MockProcessorContext implements InternalProcessorContext { +public class MockInternalProcessorContext extends MockProcessorContext implements InternalProcessorContext { private final Map restoreCallbacks = new LinkedHashMap<>(); private ProcessorNode currentNode; diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java index a75c25067a888..4ab4cb8bbbcd3 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java @@ -53,7 +53,7 @@ private MockProcessorNode(final MockProcessor mockProcessor) { } @Override - public void init(final InternalProcessorContext context) { + public void init(final InternalProcessorContext context) { super.init(context); initialized = true; } diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java index 9d22e3b78a3b7..f52134e5b493e 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -24,7 +24,7 @@ import java.util.ArrayList; import java.util.concurrent.atomic.AtomicInteger; -public class MockSourceNode extends SourceNode { +public class MockSourceNode extends SourceNode { private static final String NAME = "MOCK-SOURCE-"; private static final AtomicInteger INDEX = new AtomicInteger(1); @@ -47,7 +47,7 @@ public void process(final Record record) { } @Override - public void init(final InternalProcessorContext context) { + public void init(final InternalProcessorContext context) { super.init(context); initialized = true; } From 75a8f60a023c07054ed01194357f0d6aa7b1cff7 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 13:47:28 +0100 Subject: [PATCH 16/40] migrate test, first draft --- .../internals/AbstractProcessorContext.java | 13 +- .../internals/GlobalProcessorContextImpl.java | 2 +- .../internals/ProcessorContextImpl.java | 2 +- .../streams/TopologyTestDriverWrapper.java | 2 +- .../GlobalKTableIntegrationTest.java | 2 +- .../kstream/internals/AbstractStreamTest.java | 2 +- .../internals/GlobalKTableJoinsTest.java | 6 +- .../internals/KGroupedStreamImplTest.java | 32 ++-- .../internals/KGroupedTableImplTest.java | 10 +- .../kstream/internals/KStreamBranchTest.java | 4 +- .../kstream/internals/KStreamFilterTest.java | 4 +- .../kstream/internals/KStreamFlatMapTest.java | 2 +- .../internals/KStreamFlatMapValuesTest.java | 4 +- .../KStreamGlobalKTableJoinTest.java | 4 +- .../KStreamGlobalKTableLeftJoinTest.java | 4 +- .../kstream/internals/KStreamImplTest.java | 23 ++- .../internals/KStreamKStreamJoinTest.java | 24 +-- .../internals/KStreamKStreamLeftJoinTest.java | 35 ++-- .../internals/KStreamKTableJoinTest.java | 4 +- .../internals/KStreamKTableLeftJoinTest.java | 4 +- .../kstream/internals/KStreamMapTest.java | 2 +- .../internals/KStreamMapValuesTest.java | 2 +- .../internals/KStreamSelectKeyTest.java | 2 +- ...amSessionWindowAggregateProcessorTest.java | 159 ++++++++--------- .../KStreamSlidingWindowAggregateTest.java | 4 +- .../internals/KStreamTransformTest.java | 4 +- .../internals/KStreamTransformValuesTest.java | 26 +-- .../internals/KStreamWindowAggregateTest.java | 6 +- .../internals/KTableAggregateTest.java | 8 +- .../kstream/internals/KTableFilterTest.java | 12 +- .../kstream/internals/KTableImplTest.java | 8 +- .../internals/KTableKTableLeftJoinTest.java | 21 ++- .../kstream/internals/KTableMapKeysTest.java | 2 +- .../internals/KTableMapValuesTest.java | 14 +- .../kstream/internals/KTableReduceTest.java | 4 +- .../kstream/internals/KTableSourceTest.java | 12 +- .../internals/KTableTransformValuesTest.java | 164 +++++++++--------- .../SessionCacheFlushListenerTest.java | 1 + .../SessionWindowedKStreamImplTest.java | 6 +- .../SlidingWindowedKStreamImplTest.java | 6 +- .../TimeWindowedKStreamImplTest.java | 6 +- .../graph/GraphGraceSearchUtilTest.java | 19 +- .../suppress/KTableSuppressProcessorTest.java | 8 +- .../AbstractProcessorContextTest.java | 2 +- .../test/InternalMockProcessorContext.java | 16 +- .../org/apache/kafka/test/MockProcessor.java | 22 ++- .../apache/kafka/test/MockProcessorNode.java | 4 +- .../kafka/test/MockProcessorSupplier.java | 17 +- .../kafka/test/NoOpProcessorContext.java | 2 +- 49 files changed, 366 insertions(+), 376 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java index 37ffbdc4e6cbd..95ec0881decf2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java @@ -16,6 +16,11 @@ */ package org.apache.kafka.streams.processor.internals; +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.serialization.Serde; @@ -28,13 +33,7 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.ThreadCache; -import java.io.File; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; - -public abstract class AbstractProcessorContext implements InternalProcessorContext { +public abstract class AbstractProcessorContext implements InternalProcessorContext { private final TaskId taskId; private final String applicationId; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java index 804a30a031076..2902340ecdd0f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java @@ -34,7 +34,7 @@ import static org.apache.kafka.streams.processor.internals.AbstractReadWriteDecorator.getReadWriteStore; -public class GlobalProcessorContextImpl extends AbstractProcessorContext { +public class GlobalProcessorContextImpl extends AbstractProcessorContext { private final GlobalStateManager stateManager; private final Time time; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index a3193735a003c..74728d2957db1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -42,7 +42,7 @@ import static org.apache.kafka.streams.processor.internals.AbstractReadOnlyDecorator.getReadOnlyStore; import static org.apache.kafka.streams.processor.internals.AbstractReadWriteDecorator.getReadWriteStore; -public class ProcessorContextImpl extends AbstractProcessorContext implements RecordCollector.Supplier { +public class ProcessorContextImpl extends AbstractProcessorContext implements RecordCollector.Supplier { // the below are null for standby tasks private StreamTask streamTask; private RecordCollector collector; diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTestDriverWrapper.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTestDriverWrapper.java index e91b00723866c..12869c8b42550 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTestDriverWrapper.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTestDriverWrapper.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.processor.internals.ProcessorNode; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java index f7f71bd7bce8c..36d67c14f5fbd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java @@ -94,7 +94,7 @@ public static void closeCluster() { private String streamTopic; private GlobalKTable globalTable; private KStream stream; - private MockProcessorSupplier supplier; + private MockProcessorSupplier supplier; @Rule public TestName testName = new TestName(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java index 4bc7779f5ed6a..cfe50e266241a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java @@ -75,7 +75,7 @@ public void testToInternalValueTransformerWithKeySupplierSuppliesNewTransformers public void testShouldBeExtensible() { final StreamsBuilder builder = new StreamsBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final String topicName = "topic"; final ExtendedKStream stream = new ExtendedKStream<>(builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()))); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java index a2b8029f4be69..b088d7ecc80ec 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java @@ -58,7 +58,7 @@ public void setUp() { @Test public void shouldLeftJoinWithStream() { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream .leftJoin(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER) .process(supplier); @@ -73,7 +73,7 @@ public void shouldLeftJoinWithStream() { @Test public void shouldInnerJoinWithStream() { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream .join(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER) .process(supplier); @@ -86,7 +86,7 @@ public void shouldInnerJoinWithStream() { } private void verifyJoin(final Map> expected, - final MockProcessorSupplier supplier) { + final MockProcessorSupplier supplier) { final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 00925c4f37a58..b733959729789 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -202,7 +202,7 @@ public void shouldNotHaveInvalidStoreNameOnSlidingWindowedAggregate() { @Test public void shouldCountSlidingWindows() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(500L), ofMillis(2000L))) .count(Materialized.as("aggregate-by-key-windowed")) @@ -214,7 +214,7 @@ public void shouldCountSlidingWindows() { @Test public void shouldCountSlidingWindowsWithInternalStoreName() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(500L), ofMillis(2000L))) .count() @@ -224,7 +224,7 @@ public void shouldCountSlidingWindowsWithInternalStoreName() { doCountSlidingWindows(supplier); } - private void doCountSlidingWindows(final MockProcessorSupplier, Long> supplier) { + private void doCountSlidingWindows(final MockProcessorSupplier, Long, ?, ?> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -319,7 +319,7 @@ private void doCountSlidingWindows(final MockProcessorSupplier ))); } - private void doAggregateSessionWindows(final MockProcessorSupplier, Integer> supplier) { + private void doAggregateSessionWindows(final MockProcessorSupplier, Integer, Windowed, Integer> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -345,7 +345,7 @@ private void doAggregateSessionWindows(final MockProcessorSupplier, Integer> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Integer, Windowed, Integer> supplier = new MockProcessorSupplier<>(); final KTable, Integer> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .aggregate( @@ -363,7 +363,7 @@ public void shouldAggregateSessionWindows() { @Test public void shouldAggregateSessionWindowsWithInternalStoreName() { - final MockProcessorSupplier, Integer> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Integer, Windowed, Integer> supplier = new MockProcessorSupplier<>(); final KTable, Integer> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .aggregate( @@ -376,7 +376,7 @@ public void shouldAggregateSessionWindowsWithInternalStoreName() { doAggregateSessionWindows(supplier); } - private void doCountSessionWindows(final MockProcessorSupplier, Long> supplier) { + private void doCountSessionWindows(final MockProcessorSupplier, Long, Windowed, Long> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = @@ -403,7 +403,7 @@ private void doCountSessionWindows(final MockProcessorSupplier, @Test public void shouldCountSessionWindows() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); final KTable, Long> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .count(Materialized.as("session-store")); @@ -414,7 +414,7 @@ public void shouldCountSessionWindows() { @Test public void shouldCountSessionWindowsWithInternalStoreName() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); final KTable, Long> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .count(); @@ -423,7 +423,7 @@ public void shouldCountSessionWindowsWithInternalStoreName() { assertNull(table.queryableStoreName()); } - private void doReduceSessionWindows(final MockProcessorSupplier, String> supplier) { + private void doReduceSessionWindows(final MockProcessorSupplier, String, Windowed, String> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -449,7 +449,7 @@ private void doReduceSessionWindows(final MockProcessorSupplier @Test public void shouldReduceSessionWindows() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); final KTable, String> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .reduce((value1, value2) -> value1 + ":" + value2, Materialized.as("session-store")); @@ -460,7 +460,7 @@ public void shouldReduceSessionWindows() { @Test public void shouldReduceSessionWindowsWithInternalStoreName() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); final KTable, String> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .reduce((value1, value2) -> value1 + ":" + value2); @@ -724,7 +724,7 @@ public void shouldAggregateAndMaterializeResults() { @Test public void shouldAggregateWithDefaultSerdes() { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); groupedStream .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER) .toStream() @@ -757,7 +757,7 @@ private void processData(final TopologyTestDriver driver) { inputTopic.pipeInput("3", (String) null); } - private void doCountWindowed(final MockProcessorSupplier, Long> supplier) { + private void doCountWindowed(final MockProcessorSupplier, Long, ?, ?> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -792,7 +792,7 @@ private void doCountWindowed(final MockProcessorSupplier, Long @Test public void shouldCountWindowed() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(TimeWindows.of(ofMillis(500L))) .count(Materialized.as("aggregate-by-key-windowed")) @@ -804,7 +804,7 @@ public void shouldCountWindowed() { @Test public void shouldCountWindowedWithInternalStoreName() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(TimeWindows.of(ofMillis(500L))) .count() diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java index 130e299ff8835..da56674b8bd57 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -126,8 +126,8 @@ public void shouldNotAllowInvalidStoreNameOnReduce() { Materialized.as(INVALID_STORE_NAME))); } - private MockProcessorSupplier getReducedResults(final KTable inputKTable) { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private MockProcessorSupplier getReducedResults(final KTable inputKTable) { + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); inputKTable .toStream() .process(supplier); @@ -172,7 +172,7 @@ public void shouldReduce() { MockReducer.INTEGER_SUBTRACTOR, Materialized.as("reduced")); - final MockProcessorSupplier supplier = getReducedResults(reduced); + final MockProcessorSupplier supplier = getReducedResults(reduced); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { assertReduced(supplier.theCapturedProcessor().lastValueAndTimestampPerKey(), topic, driver); assertEquals(reduced.queryableStoreName(), "reduced"); @@ -194,7 +194,7 @@ public void shouldReduceWithInternalStoreName() { .groupBy(intProjection) .reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR); - final MockProcessorSupplier supplier = getReducedResults(reduced); + final MockProcessorSupplier supplier = getReducedResults(reduced); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { assertReduced(supplier.theCapturedProcessor().lastValueAndTimestampPerKey(), topic, driver); assertNull(reduced.queryableStoreName()); @@ -219,7 +219,7 @@ public void shouldReduceAndMaterializeResults() { .withKeySerde(Serdes.String()) .withValueSerde(Serdes.Integer())); - final MockProcessorSupplier supplier = getReducedResults(reduced); + final MockProcessorSupplier supplier = getReducedResults(reduced); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { assertReduced(supplier.theCapturedProcessor().lastValueAndTimestampPerKey(), topic, driver); { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index 1813522bcce2c..c947338b2d6c0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -59,7 +59,7 @@ public void testKStreamBranch() { assertEquals(3, branches.length); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); for (final KStream branch : branches) { branch.process(supplier); } @@ -71,7 +71,7 @@ public void testKStreamBranch() { } } - final List> processors = supplier.capturedProcessors(3); + final List> processors = supplier.capturedProcessors(3); assertEquals(3, processors.get(0).processed().size()); assertEquals(1, processors.get(1).processed().size()); assertEquals(2, processors.get(2).processed().size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index c6c2c7a7c37fb..4ecc24079ccad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -46,7 +46,7 @@ public void testFilter() { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; final KStream stream; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.filter(isMultipleOfThree).process(supplier); @@ -67,7 +67,7 @@ public void testFilterNot() { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; final KStream stream; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.filterNot(isMultipleOfThree).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index e27754a301df8..2597a663ea3d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -58,7 +58,7 @@ public void testFlatMap() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream; - final MockProcessorSupplier supplier; + final MockProcessorSupplier supplier; supplier = new MockProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index a84fdcc8024eb..b2ae36509b41b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -56,7 +56,7 @@ public void testFlatMapValues() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream.flatMapValues(mapper).process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -92,7 +92,7 @@ public void testFlatMapValuesWithKeys() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream.flatMapValues(mapper).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java index abebd52141973..c2212555b4141 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java @@ -52,7 +52,7 @@ public class KStreamGlobalKTableJoinTest { private final int[] expectedKeys = {0, 1, 2, 3}; private TopologyTestDriver driver; - private MockProcessor processor; + private MockProcessor processor; private StreamsBuilder builder; @Before @@ -63,7 +63,7 @@ public void setUp() { final GlobalKTable table; // value of stream optionally contains key of table final KeyValueMapper keyMapper; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final Consumed streamConsumed = Consumed.with(Serdes.Integer(), Serdes.String()); final Consumed tableConsumed = Consumed.with(Serdes.String(), Serdes.String()); stream = builder.stream(streamTopic, streamConsumed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java index 2b2541ee4ebb3..f0cc2656f9a9c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java @@ -51,7 +51,7 @@ public class KStreamGlobalKTableLeftJoinTest { private final String globalTableTopic = "globalTableTopic"; private final int[] expectedKeys = {0, 1, 2, 3}; - private MockProcessor processor; + private MockProcessor processor; private TopologyTestDriver driver; private StreamsBuilder builder; @@ -63,7 +63,7 @@ public void setUp() { final GlobalKTable table; // value of stream optionally contains key of table final KeyValueMapper keyMapper; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final Consumed streamConsumed = Consumed.with(Serdes.Integer(), Serdes.String()); final Consumed tableConsumed = Consumed.with(Serdes.String(), Serdes.String()); stream = builder.stream(streamTopic, streamConsumed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index ce58724048405..6787e2390e4d9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -55,7 +55,6 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.SourceNode; @@ -103,11 +102,11 @@ public class KStreamImplTest { private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String()); - private final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + private final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); private final TransformerSupplier> transformerSupplier = () -> new Transformer>() { @Override - public void init(final ProcessorContext context) {} + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {} @Override public KeyValue transform(final String key, final String value) { @@ -120,7 +119,7 @@ public void close() {} private final TransformerSupplier>> flatTransformerSupplier = () -> new Transformer>>() { @Override - public void init(final ProcessorContext context) {} + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {} @Override public Iterable> transform(final String key, final String value) { @@ -133,7 +132,7 @@ public void close() {} private final ValueTransformerSupplier valueTransformerSupplier = () -> new ValueTransformer() { @Override - public void init(final ProcessorContext context) {} + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {} @Override public String transform(final String value) { @@ -146,7 +145,7 @@ public void close() {} private final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier = () -> new ValueTransformerWithKey() { @Override - public void init(final ProcessorContext context) {} + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {} @Override public String transform(final String key, final String value) { @@ -159,7 +158,7 @@ public void close() {} private final ValueTransformerSupplier> flatValueTransformerSupplier = () -> new ValueTransformer>() { @Override - public void init(final ProcessorContext context) {} + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {} @Override public Iterable transform(final String value) { @@ -1504,7 +1503,7 @@ public void shouldSendDataToDynamicTopics() { inputTopic.pipeInput("a", "v2"); inputTopic.pipeInput("b", "v1"); } - final List> mockProcessors = processorSupplier.capturedProcessors(2); + final List> mockProcessors = processorSupplier.capturedProcessors(2); assertThat(mockProcessors.get(0).processed(), equalTo(asList(new KeyValueTimestamp<>("a", "v1", 0), new KeyValueTimestamp<>("a", "v2", 0)))); assertThat(mockProcessors.get(1).processed(), equalTo(Collections.singletonList(new KeyValueTimestamp<>("b", "v1", 0)))); @@ -2382,7 +2381,7 @@ public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueWithKeySupp public void shouldNotAllowNullProcessSupplierOnProcess() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null)); + () -> testStream.process(null)); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2390,7 +2389,7 @@ public void shouldNotAllowNullProcessSupplierOnProcess() { public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null, "storeName")); + () -> testStream.process(null, "storeName")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2398,7 +2397,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null, Named.as("processor"))); + () -> testStream.process(null, Named.as("processor"))); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2406,7 +2405,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamedAndStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null, Named.as("processor"), "stateStore")); + () -> testStream.process(null, Named.as("processor"), "stateStore")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index 78340d7aea5fd..e94d49948e9e7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -346,7 +346,7 @@ private void runJoin(final StreamJoined streamJoined, final KStream left = builder.stream("left", Consumed.with(Serdes.String(), Serdes.Integer())); final KStream right = builder.stream("right", Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final KStream joinedStream; joinedStream = left.join( @@ -362,7 +362,7 @@ private void runJoin(final StreamJoined streamJoined, driver.createInputTopic("left", new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopicRight = driver.createInputTopic("right", new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); inputTopicLeft.pipeInput("A", 1, 1L); inputTopicLeft.pipeInput("B", 1, 2L); @@ -384,7 +384,7 @@ public void testJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); joined = stream1.join( @@ -405,7 +405,7 @@ public void testJoin() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty // w1 = {} @@ -495,7 +495,7 @@ public void testOuterJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -516,7 +516,7 @@ public void testOuterJoin() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty; this should produce two items // w1 = {} @@ -609,7 +609,7 @@ public void testWindowing() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -631,7 +631,7 @@ public void testWindowing() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); long time = 0L; // push two items to the primary stream; the other window is empty; this should produce no items @@ -1137,7 +1137,7 @@ public void testAsymmetricWindowingAfter() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -1161,7 +1161,7 @@ public void testAsymmetricWindowingAfter() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); long time = 1000L; // push four items with increasing timestamps to the primary stream; the other window is empty; this should produce no items @@ -1388,7 +1388,7 @@ public void testAsymmetricWindowingBefore() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -1411,7 +1411,7 @@ public void testAsymmetricWindowingBefore() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); long time = 1000L; // push four items with increasing timestamps to the primary stream; the other window is empty; this should produce no items diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index fafda4ade89f2..504f3adf43f0b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -16,17 +16,27 @@ */ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static org.junit.Assert.assertEquals; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Properties; +import java.util.Set; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.TopologyWrapper; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.kstream.StreamJoined; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; @@ -34,17 +44,6 @@ import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.Properties; -import java.util.Set; - -import static java.time.Duration.ofMillis; -import static org.junit.Assert.assertEquals; - public class KStreamKStreamLeftJoinTest { private final static KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0]; @@ -62,7 +61,7 @@ public void testLeftJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -84,7 +83,7 @@ public void testLeftJoin() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty // w1 {} @@ -155,7 +154,7 @@ public void testWindowing() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -177,7 +176,7 @@ public void testWindowing() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockProcessor processor = supplier.theCapturedProcessor(); final long time = 0L; // push two items to the primary stream; the other window is empty; this should produce two left-join items @@ -207,7 +206,7 @@ public void testWindowing() { private void testUpperWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, - final MockProcessor processor) { + final MockProcessor processor) { long time; final TestInputTopic inputTopic1 = @@ -331,7 +330,7 @@ private void testUpperWindowBound(final int[] expectedKeys, private void testLowerWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, - final MockProcessor processor) { + final MockProcessor processor) { long time; final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer()); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java index 80b776eb10811..e821d97b0ab71 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java @@ -62,11 +62,11 @@ public class KStreamKTableJoinTest { private TestInputTopic inputTableTopic; private final int[] expectedKeys = {0, 1, 2, 3}; - private MockProcessor processor; + private MockProcessor processor; private TopologyTestDriver driver; private StreamsBuilder builder; private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); @Before public void setUp() { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index 30e0ff86e11bc..996e250412373 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -56,7 +56,7 @@ public class KStreamKTableLeftJoinTest { private final int[] expectedKeys = {0, 1, 2, 3}; private TopologyTestDriver driver; - private MockProcessor processor; + private MockProcessor processor; private StreamsBuilder builder; @Before @@ -66,7 +66,7 @@ public void setUp() { final KStream stream; final KTable table; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); stream = builder.stream(streamTopic, consumed); table = builder.table(tableTopic, consumed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 623dde50c0c9e..94b8066feeeb7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -45,7 +45,7 @@ public void testMap() { final String topicName = "topic"; final int[] expectedKeys = new int[] {0, 1, 2, 3}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.map((key, value) -> KeyValue.pair(value, key)).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index 057294864babc..c83ba5d403793 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -36,7 +36,7 @@ public class KStreamMapValuesTest { private final String topicName = "topic"; - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java index a7546e900796a..66d84e9b3c85a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java @@ -57,7 +57,7 @@ public void testSelectKey() { final KStream stream = builder.stream(topicName, Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream.selectKey((key, value) -> keyMap.get(value)).process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index a836c565d54b9..c2c38e17fcca2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -16,6 +16,22 @@ */ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serdes; @@ -31,13 +47,14 @@ import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; -import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.ToInternal; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; @@ -52,23 +69,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static java.time.Duration.ofMillis; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; -import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.greaterThan; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - public class KStreamSessionWindowAggregateProcessorTest { private static final long GAP_MS = 5 * 60 * 1000L; @@ -88,7 +88,7 @@ public class KStreamSessionWindowAggregateProcessorTest { sessionMerger); private final List, Change>> results = new ArrayList<>(); - private final Processor processor = sessionAggregator.get(); + private final Processor processor = sessionAggregator.get(); private SessionStore sessionStore; private InternalMockProcessorContext context; private Metrics metrics; @@ -144,10 +144,8 @@ public void closeStore() { @Test public void shouldCreateSingleSessionWhenWithinGap() { - context.setTime(0); - processor.process("john", "first"); - context.setTime(500); - processor.process("john", "second"); + processor.process(new Record<>("john", "first", 0L)); + processor.process(new Record<>("john", "second", 500L)); final KeyValueIterator, Long> values = sessionStore.findSessions("john", 0, 2000); @@ -157,20 +155,17 @@ public void shouldCreateSingleSessionWhenWithinGap() { @Test public void shouldMergeSessions() { - context.setTime(0); final String sessionId = "mel"; - processor.process(sessionId, "first"); + processor.process(new Record<>(sessionId, "first", 0L)); assertTrue(sessionStore.findSessions(sessionId, 0, 0).hasNext()); // move time beyond gap - context.setTime(GAP_MS + 1); - processor.process(sessionId, "second"); + processor.process(new Record<>(sessionId, "second", GAP_MS + 1)); assertTrue(sessionStore.findSessions(sessionId, GAP_MS + 1, GAP_MS + 1).hasNext()); // should still exist as not within gap assertTrue(sessionStore.findSessions(sessionId, 0, 0).hasNext()); // move time back - context.setTime(GAP_MS / 2); - processor.process(sessionId, "third"); + processor.process(new Record<>(sessionId, "third", GAP_MS / 2)); final KeyValueIterator, Long> iterator = sessionStore.findSessions(sessionId, 0, GAP_MS + 1); @@ -182,9 +177,8 @@ public void shouldMergeSessions() { @Test public void shouldUpdateSessionIfTheSameTime() { - context.setTime(0); - processor.process("mel", "first"); - processor.process("mel", "second"); + processor.process(new Record<>("mel", "first", 0L)); + processor.process(new Record<>("mel", "second", 0L)); final KeyValueIterator, Long> iterator = sessionStore.findSessions("mel", 0, 0); assertEquals(Long.valueOf(2L), iterator.next().value); @@ -195,15 +189,14 @@ public void shouldUpdateSessionIfTheSameTime() { public void shouldHaveMultipleSessionsForSameIdWhenTimestampApartBySessionGap() { final String sessionId = "mel"; long time = 0; - context.setTime(time); - processor.process(sessionId, "first"); - context.setTime(time += GAP_MS + 1); - processor.process(sessionId, "second"); - processor.process(sessionId, "second"); - context.setTime(time += GAP_MS + 1); - processor.process(sessionId, "third"); - processor.process(sessionId, "third"); - processor.process(sessionId, "third"); + processor.process(new Record<>(sessionId, "first", time)); + time += GAP_MS + 1; + processor.process(new Record<>(sessionId, "second", time)); + processor.process(new Record<>(sessionId, "second", time)); + time += GAP_MS + 1; + processor.process(new Record<>(sessionId, "third", time)); + processor.process(new Record<>(sessionId, "third", time)); + processor.process(new Record<>(sessionId, "third", time)); sessionStore.flush(); assertEquals( @@ -228,16 +221,14 @@ public void shouldHaveMultipleSessionsForSameIdWhenTimestampApartBySessionGap() @Test public void shouldRemoveMergedSessionsFromStateStore() { - context.setTime(0); - processor.process("a", "1"); + processor.process(new Record<>("a", "1", 0)); // first ensure it is in the store final KeyValueIterator, Long> a1 = sessionStore.findSessions("a", 0, 0); assertEquals(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 1L), a1.next()); - context.setTime(100); - processor.process("a", "2"); + processor.process(new Record<>("a", "2", 100)); // a1 from above should have been removed // should have merged session in store final KeyValueIterator, Long> a2 = @@ -249,18 +240,15 @@ public void shouldRemoveMergedSessionsFromStateStore() { @Test public void shouldHandleMultipleSessionsAndMerging() { context.setTime(0); - processor.process("a", "1"); - processor.process("b", "1"); - processor.process("c", "1"); - processor.process("d", "1"); - context.setTime(GAP_MS / 2); - processor.process("d", "2"); - context.setTime(GAP_MS + 1); - processor.process("a", "2"); - processor.process("b", "2"); - context.setTime(GAP_MS + 1 + GAP_MS / 2); - processor.process("a", "3"); - processor.process("c", "3"); + processor.process(new Record<>("a", "1", 0)); + processor.process(new Record<>("b", "1", 0)); + processor.process(new Record<>("c", "1", 0)); + processor.process(new Record<>("d", "1", 0)); + processor.process(new Record<>("d", "2", GAP_MS / 2)); + processor.process(new Record<>("a", "2", GAP_MS + 1)); + processor.process(new Record<>("b", "2", GAP_MS + 1)); + processor.process(new Record<>("a", "3", GAP_MS + 1 + GAP_MS / 2)); + processor.process(new Record<>("c", "3", GAP_MS + 1 + GAP_MS / 2)); sessionStore.flush(); @@ -301,15 +289,15 @@ public void shouldHandleMultipleSessionsAndMerging() { @Test public void shouldGetAggregatedValuesFromValueGetter() { - final KTableValueGetter, Long> getter = sessionAggregator.view().get(); + final KTableValueAndTimestampGetter, Long> getter = sessionAggregator + .view().get(); getter.init(context); - context.setTime(0); - processor.process("a", "1"); - context.setTime(GAP_MS + 1); - processor.process("a", "1"); - processor.process("a", "2"); + processor.process(new Record<>("a", "1", 0)); + processor.process(new Record<>("a", "1", GAP_MS + 1)); + processor.process(new Record<>("a", "2", GAP_MS + 1)); final long t0 = getter.get(new Windowed<>("a", new SessionWindow(0, 0))).value(); - final long t1 = getter.get(new Windowed<>("a", new SessionWindow(GAP_MS + 1, GAP_MS + 1))).value(); + final long t1 = getter.get(new Windowed<>("a", new SessionWindow(GAP_MS + 1, GAP_MS + 1))) + .value(); assertEquals(1L, t0); assertEquals(2L, t1); } @@ -319,10 +307,9 @@ public void shouldImmediatelyForwardNewSessionWhenNonCachedStore() { initStore(false); processor.init(context); - context.setTime(0); - processor.process("a", "1"); - processor.process("b", "1"); - processor.process("c", "1"); + processor.process(new Record<>("a", "1", 0)); + processor.process(new Record<>("b", "1", 0)); + processor.process(new Record<>("c", "1", 0)); assertEquals( Arrays.asList( @@ -348,10 +335,8 @@ public void shouldImmediatelyForwardRemovedSessionsWhenMerging() { initStore(false); processor.init(context); - context.setTime(0); - processor.process("a", "1"); - context.setTime(5); - processor.process("a", "1"); + processor.process(new Record<>("a", "1", 0)); + processor.process(new Record<>("a", "1", 5)); assertEquals( Arrays.asList( new KeyValueTimestamp<>( @@ -392,7 +377,7 @@ private void shouldLogAndMeterWhenSkippingNullKeyWithBuiltInMetrics(final String try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KStreamSessionWindowAggregate.class)) { - processor.process(null, "1"); + processor.process(new Record<>(null, "1", 0)); assertThat( appender.getMessages(), @@ -425,7 +410,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithZeroGraceWithBuiltInMetri private void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace(final String builtInMetricsVersion) { final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); - final Processor processor = new KStreamSessionWindowAggregate<>( + final Processor processor = new KStreamSessionWindowAggregate<>( SessionWindows.with(ofMillis(10L)).grace(ofMillis(0L)), STORE_NAME, initializer, @@ -437,22 +422,22 @@ private void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace(final String b // dummy record to establish stream time = 0 context.setRecordContext(new ProcessorRecordContext(0, -2, -3, "topic", null)); - processor.process("dummy", "dummy"); + processor.process(new Record<>("dummy", "dummy", 0)); // record arrives on time, should not be skipped context.setRecordContext(new ProcessorRecordContext(0, -2, -3, "topic", null)); - processor.process("OnTime1", "1"); + processor.process(new Record<>("OnTime1", "1", 0)); // dummy record to advance stream time = 1 context.setRecordContext(new ProcessorRecordContext(1, -2, -3, "topic", null)); - processor.process("dummy", "dummy"); + processor.process(new Record<>("dummy", "dummy", 1)); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KStreamSessionWindowAggregate.class)) { // record is late context.setRecordContext(new ProcessorRecordContext(0, -2, -3, "topic", null)); - processor.process("Late1", "1"); + processor.process(new Record<>("Late1", "1", 0)); assertThat( appender.getMessages(), @@ -523,7 +508,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGraceWithBuiltInMe private void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace(final String builtInMetricsVersion) { final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); - final Processor processor = new KStreamSessionWindowAggregate<>( + final Processor processor = new KStreamSessionWindowAggregate<>( SessionWindows.with(ofMillis(10L)).grace(ofMillis(1L)), STORE_NAME, initializer, @@ -538,27 +523,27 @@ private void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace(final Strin // dummy record to establish stream time = 0 context.setRecordContext(new ProcessorRecordContext(0, -2, -3, "topic", null)); - processor.process("dummy", "dummy"); + processor.process(new Record<>("dummy", "dummy", 0)); // record arrives on time, should not be skipped context.setRecordContext(new ProcessorRecordContext(0, -2, -3, "topic", null)); - processor.process("OnTime1", "1"); + processor.process(new Record<>("OnTime1", "1", 0)); // dummy record to advance stream time = 1 context.setRecordContext(new ProcessorRecordContext(1, -2, -3, "topic", null)); - processor.process("dummy", "dummy"); + processor.process(new Record<>("dummy", "dummy", 1)); // delayed record arrives on time, should not be skipped context.setRecordContext(new ProcessorRecordContext(0, -2, -3, "topic", null)); - processor.process("OnTime2", "1"); + processor.process(new Record<>("OnTime2", "1", 0)); // dummy record to advance stream time = 2 context.setRecordContext(new ProcessorRecordContext(2, -2, -3, "topic", null)); - processor.process("dummy", "dummy"); + processor.process(new Record<>("dummy", "dummy", 2)); // delayed record arrives late context.setRecordContext(new ProcessorRecordContext(0, -2, -3, "topic", null)); - processor.process("Late1", "1"); + processor.process(new Record<>("Late1", "1", 0)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java index 0bb2c01226348..3880a545d25dc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java @@ -117,7 +117,7 @@ public void testAggregateSmallInput() { MockAggregator.TOSTRING_ADDER, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); table.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = @@ -173,7 +173,7 @@ public void testReduceSmallInput() { MockReducer.STRING_ADDER, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); table.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index 20b1d3a896676..b821ea66105f9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -73,7 +73,7 @@ public void close() { } final int[] expectedKeys = {1, 10, 100, 1000}; - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); final KStream stream = builder.stream(TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer())); stream.transform(transformerSupplier).process(processor); @@ -134,7 +134,7 @@ public void close() { } final int[] expectedKeys = {1, 10, 100, 1000}; - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); final KStream stream = builder.stream(TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer())); stream.transform(transformerSupplier).process(processor); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index 96f80ff93aadc..56ab2c6c2b009 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -49,7 +49,7 @@ @RunWith(EasyMockRunner.class) public class KStreamTransformValuesTest { private final String topicName = "topic"; - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.Integer()); @Mock(MockType.NICE) private ProcessorContext context; @@ -137,16 +137,16 @@ public void close() { } assertArrayEquals(expected, supplier.theCapturedProcessor().processed().toArray()); } - - @SuppressWarnings("unchecked") - @Test - public void shouldInitializeTransformerWithForwardDisabledProcessorContext() { - final NoOpValueTransformerWithKeySupplier transformer = new NoOpValueTransformerWithKeySupplier<>(); - final KStreamTransformValues transformValues = new KStreamTransformValues<>(transformer); - final Processor processor = transformValues.get(); - - processor.init(context); - - assertThat(transformer.context, isA((Class) ForwardingDisabledProcessorContext.class)); - } +// TODO +// @SuppressWarnings("unchecked") +// @Test +// public void shouldInitializeTransformerWithForwardDisabledProcessorContext() { +// final NoOpValueTransformerWithKeySupplier transformer = new NoOpValueTransformerWithKeySupplier<>(); +// final KStreamTransformValues transformValues = new KStreamTransformValues<>(transformer); +// final Processor processor = transformValues.get(); +// +// processor.init(context); +// +// assertThat(transformer.context, isA((Class) ForwardingDisabledProcessorContext.class)); +// } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 4270f1afd1c11..71ca7cc590f98 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -79,7 +79,7 @@ public void testAggBasic() { .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -157,7 +157,7 @@ public void testJoin() { .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); final KTable, String> table2 = builder @@ -180,7 +180,7 @@ public void testJoin() { inputTopic1.pipeInput("D", "4", 3L); inputTopic1.pipeInput("A", "1", 9L); - final List, String>> processors = supplier.capturedProcessors(3); + List, String, ?, ?>> processors = supplier.capturedProcessors(3); processors.get(0).checkAndClearProcessResult( new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(0, 10)), "0+1", 0), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index b188853d27ecb..a1d322c144b69 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -53,7 +53,7 @@ public class KTableAggregateTest { private final Serde stringSerde = Serdes.String(); private final Consumed consumed = Consumed.with(stringSerde, stringSerde); private final Grouped stringSerialized = Grouped.with(stringSerde, stringSerde); - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); private final static Properties CONFIG = mkProperties(mkMap( mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory("kafka-test").getAbsolutePath()))); @@ -168,7 +168,7 @@ public void testAggRepartition() { private static void testCountHelper(final StreamsBuilder builder, final String input, - final MockProcessorSupplier supplier) { + final MockProcessorSupplier supplier) { try ( final TopologyTestDriver driver = new TopologyTestDriver( builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { @@ -228,7 +228,7 @@ public void testCountWithInternalStore() { public void testRemoveOldBeforeAddNew() { final StreamsBuilder builder = new StreamsBuilder(); final String input = "count-test-input"; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); builder .table(input, consumed) @@ -252,7 +252,7 @@ public void testRemoveOldBeforeAddNew() { final TestInputTopic inputTopic = driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor proc = supplier.theCapturedProcessor(); + final MockProcessor proc = supplier.theCapturedProcessor(); inputTopic.pipeInput("11", "A", 10L); inputTopic.pipeInput("12", "B", 8L); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 7f772e7c693fe..ec0ffd2eaad2b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -70,7 +70,7 @@ private void doTestKTable(final StreamsBuilder builder, final KTable table2, final KTable table3, final String topic) { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); table3.toStream().process(supplier); @@ -85,7 +85,7 @@ private void doTestKTable(final StreamsBuilder builder, inputTopic.pipeInput("B", null, 15L); } - final List> processors = supplier.capturedProcessors(2); + final List> processors = supplier.capturedProcessors(2); processors.get(0).checkAndClearProcessResult(new KeyValueTimestamp<>("A", null, 10), new KeyValueTimestamp<>("B", 2, 5), @@ -140,8 +140,8 @@ private void doTestValueGetter(final StreamsBuilder builder, final Topology topology = builder.build(); - final KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); - final KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); + final KTableValueAndTimestampGetterSupplier getterSupplier2 = table2.valueAndTimestampGetterSupplier(); + final KTableValueAndTimestampGetterSupplier getterSupplier3 = table3.valueAndTimestampGetterSupplier(); final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology); topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames()); @@ -151,8 +151,8 @@ private void doTestValueGetter(final StreamsBuilder builder, final TestInputTopic inputTopic = driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final KTableValueGetter getter2 = getterSupplier2.get(); - final KTableValueGetter getter3 = getterSupplier3.get(); + final KTableValueAndTimestampGetter getter2 = getterSupplier2.get(); + final KTableValueAndTimestampGetter getter3 = getterSupplier3.get(); getter2.init(driver.setCurrentNodeForProcessorContext(table2.name)); getter3.init(driver.setCurrentNodeForProcessorContext(table3.name)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 348750a725975..4f4045df9f1f2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -91,7 +91,7 @@ public void testKTable() { final KTable table1 = builder.table(topic1, consumed); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); @@ -115,7 +115,7 @@ public void testKTable() { inputTopic.pipeInput("A", "06", 8L); } - final List> processors = supplier.capturedProcessors(4); + final List> processors = supplier.capturedProcessors(4); assertEquals(asList( new KeyValueTimestamp<>("A", "01", 5), new KeyValueTimestamp<>("B", "02", 100), @@ -159,7 +159,7 @@ public void testMaterializedKTable() { final KTable table1 = builder.table(topic1, consumed, Materialized.as("fred")); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); @@ -183,7 +183,7 @@ public void testMaterializedKTable() { inputTopic.pipeInput("A", "06", 8L); } - final List> processors = supplier.capturedProcessors(4); + final List> processors = supplier.capturedProcessors(4); assertEquals(asList( new KeyValueTimestamp<>("A", "01", 5), new KeyValueTimestamp<>("B", "02", 100), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index c1bc7fe7df005..6a57bee452c53 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -32,8 +32,9 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.streams.processor.MockProcessorContext; -import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.api.MockProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.test.TestRecord; @@ -523,23 +524,25 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKeyWithBuiltInMetricsVer shouldLogAndMeterSkippedRecordsDueToNullLeftKey(StreamsConfig.METRICS_0100_TO_24); } + @SuppressWarnings("unchecked") private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtInMetricsVersion) { final StreamsBuilder builder = new StreamsBuilder(); - @SuppressWarnings("unchecked") - final Processor> join = new KTableKTableLeftJoin<>( - (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())), - (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())), + final Processor, String, Change> join = new KTableKTableLeftJoin<>( + (KTableImpl) builder + .table("left", Consumed.with(Serdes.String(), Serdes.String())), + (KTableImpl) builder + .table("right", Consumed.with(Serdes.String(), Serdes.String())), null ).get(); props.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); - final MockProcessorContext context = new MockProcessorContext(props); - context.setRecordMetadata("left", -1, -2, null, -3); + final MockProcessorContext> context = new MockProcessorContext<>(props); + context.setRecordMetadata("left", -1, -2); join.init(context); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KTableKTableLeftJoin.class)) { - join.process(null, new Change<>("new", "old")); + join.process(new Record<>(null, new Change<>("new", "old"), 0L)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java index 548ac17d5622e..d7500d6171a11 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java @@ -59,7 +59,7 @@ public void testMapKeysConvertingToStream() { final int[] originalKeys = new int[] {1, 2, 3}; final String[] values = new String[] {"V_ONE", "V_TWO", "V_THREE"}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); convertedStream.process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 384d57902187d..c1c6993f9558a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -56,7 +56,7 @@ public class KTableMapValuesTest { private void doTestKTable(final StreamsBuilder builder, final String topic1, - final MockProcessorSupplier supplier) { + final MockProcessorSupplier supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); @@ -79,7 +79,7 @@ public void testKTable() { final KTable table1 = builder.table(topic1, consumed); final KTable table2 = table1.mapValues(value -> value.charAt(0) - 48); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); doTestKTable(builder, topic1, supplier); @@ -97,7 +97,7 @@ public void testQueryableKTable() { Materialized.>as("anyName") .withValueSerde(Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); doTestKTable(builder, topic1, supplier); @@ -110,8 +110,8 @@ private void doTestValueGetter(final StreamsBuilder builder, final Topology topology = builder.build(); - final KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); - final KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); + final KTableValueAndTimestampGetterSupplier getterSupplier2 = table2.valueAndTimestampGetterSupplier(); + final KTableValueAndTimestampGetterSupplier getterSupplier3 = table3.valueAndTimestampGetterSupplier(); final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology); topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames()); @@ -120,8 +120,8 @@ private void doTestValueGetter(final StreamsBuilder builder, try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(builder.build(), props)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final KTableValueGetter getter2 = getterSupplier2.get(); - final KTableValueGetter getter3 = getterSupplier3.get(); + final KTableValueAndTimestampGetter getter2 = getterSupplier2.get(); + final KTableValueAndTimestampGetter getter3 = getterSupplier3.get(); getter2.init(driver.setCurrentNodeForProcessorContext(table2.name)); getter3.init(driver.setCurrentNodeForProcessorContext(table3.name)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java index b360151f444e0..008effebb5ba8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -38,7 +38,7 @@ public class KTableReduceTest { public void shouldAddAndSubtract() { final InternalMockProcessorContext context = new InternalMockProcessorContext(); - final Processor>> reduceProcessor = + final Processor>, ?, ?> reduceProcessor = new KTableReduce>( "myStore", this::unionNotNullArgs, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 02590f6063c18..4d5611576bbd2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -68,7 +68,7 @@ public void testKTable() { final KTable table1 = builder.table(topic1, Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -208,17 +208,17 @@ public void kTableShouldLogAndMeterOnSkippedRecords() { } } + @SuppressWarnings("unchecked") @Test public void testValueGetter() { final StreamsBuilder builder = new StreamsBuilder(); final String topic1 = "topic1"; - @SuppressWarnings("unchecked") final KTableImpl table1 = (KTableImpl) builder.table(topic1, stringConsumed, Materialized.as("store")); final Topology topology = builder.build(); - final KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); + final KTableValueAndTimestampGetterSupplier getterSupplier1 = table1.valueAndTimestampGetterSupplier(); final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology); topologyBuilder.connectProcessorAndStateStores(table1.name, getterSupplier1.storeNames()); @@ -232,7 +232,7 @@ public void testValueGetter() { Instant.ofEpochMilli(0L), Duration.ZERO ); - final KTableValueGetter getter1 = getterSupplier1.get(); + final KTableValueAndTimestampGetter getter1 = getterSupplier1.get(); getter1.init(driver.setCurrentNodeForProcessorContext(table1.name)); inputTopic1.pipeInput("A", "01", 10L); @@ -265,12 +265,12 @@ public void testValueGetter() { } } + @SuppressWarnings("unchecked") @Test public void testNotSendingOldValue() { final StreamsBuilder builder = new StreamsBuilder(); final String topic1 = "topic1"; - @SuppressWarnings("unchecked") final KTableImpl table1 = (KTableImpl) builder.table(topic1, stringConsumed); @@ -318,12 +318,12 @@ public void testNotSendingOldValue() { } } + @SuppressWarnings("unchecked") @Test public void testSendingOldValue() { final StreamsBuilder builder = new StreamsBuilder(); final String topic1 = "topic1"; - @SuppressWarnings("unchecked") final KTableImpl table1 = (KTableImpl) builder.table(topic1, stringConsumed); table1.enableSendingOldValues(true); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java index e217a66ee507c..67b7df1c5de89 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java @@ -32,9 +32,7 @@ import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; @@ -80,16 +78,16 @@ public class KTableTransformValuesTest { private static final Consumed CONSUMED = Consumed.with(Serdes.String(), Serdes.String()); private TopologyTestDriver driver; - private MockProcessorSupplier capture; + private MockProcessorSupplier capture; private StreamsBuilder builder; @Mock(MockType.NICE) private KTableImpl parent; @Mock(MockType.NICE) private InternalProcessorContext context; @Mock(MockType.NICE) - private KTableValueGetterSupplier parentGetterSupplier; + private KTableValueAndTimestampGetterSupplier parentGetterSupplier; @Mock(MockType.NICE) - private KTableValueGetter parentGetter; + private KTableValueAndTimestampGetter parentGetter; @Mock(MockType.NICE) private TimestampedKeyValueStore stateStore; @Mock(MockType.NICE) @@ -126,7 +124,7 @@ public void shouldThrowOnGetIfSupplierReturnsNull() { @Test public void shouldThrowOnViewGetIfSupplierReturnsNull() { - final KTableValueGetterSupplier view = + final KTableValueAndTimestampGetterSupplier view = new KTableTransformValues<>(parent, new NullSupplier(), null).view(); try { @@ -137,56 +135,58 @@ public void shouldThrowOnViewGetIfSupplierReturnsNull() { } } - @SuppressWarnings("unchecked") - @Test - public void shouldInitializeTransformerWithForwardDisabledProcessorContext() { - final NoOpValueTransformerWithKeySupplier transformer = new NoOpValueTransformerWithKeySupplier<>(); - final KTableTransformValues transformValues = - new KTableTransformValues<>(parent, transformer, null); - final Processor> processor = transformValues.get(); - - processor.init(context); - - assertThat(transformer.context, isA((Class) ForwardingDisabledProcessorContext.class)); - } - - @Test - public void shouldNotSendOldValuesByDefault() { - final KTableTransformValues transformValues = - new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); - - final Processor> processor = transformValues.get(); - processor.init(context); - - context.forward("Key", new Change<>("Key->newValue!", null)); - expectLastCall(); - replay(context); - - processor.process("Key", new Change<>("newValue", "oldValue")); - - verify(context); - } - - @Test - public void shouldSendOldValuesIfConfigured() { - final KTableTransformValues transformValues = - new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); - - expect(parent.enableSendingOldValues(true)).andReturn(true); - replay(parent); - - transformValues.enableSendingOldValues(true); - final Processor> processor = transformValues.get(); - processor.init(context); - - context.forward("Key", new Change<>("Key->newValue!", "Key->oldValue!")); - expectLastCall(); - replay(context); - - processor.process("Key", new Change<>("newValue", "oldValue")); - - verify(context); - } +//TODO +// @SuppressWarnings("unchecked") +// @Test +// public void shouldInitializeTransformerWithForwardDisabledProcessorContext() { +// final NoOpValueTransformerWithKeySupplier transformer = new NoOpValueTransformerWithKeySupplier<>(); +// final KTableTransformValues transformValues = +// new KTableTransformValues<>(parent, transformer, null); +// final Processor> processor = transformValues.get(); +// +// processor.init(context); +// +// assertThat(transformer.context, isA((Class) ForwardingDisabledProcessorContext.class)); +// } + +// TODO +// @Test +// public void shouldNotSendOldValuesByDefault() { +// final KTableTransformValues transformValues = +// new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); +// +// final Processor> processor = transformValues.get(); +// processor.init(context); +// +// context.forward("Key", new Change<>("Key->newValue!", null)); +// expectLastCall(); +// replay(context); +// +// processor.process("Key", new Change<>("newValue", "oldValue")); +// +// verify(context); +// } + +// @Test +// public void shouldSendOldValuesIfConfigured() { +// final KTableTransformValues transformValues = +// new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); +// +// expect(parent.enableSendingOldValues(true)).andReturn(true); +// replay(parent); +// +// transformValues.enableSendingOldValues(true); +// final Processor> processor = transformValues.get(); +// processor.init(context); +// +// context.forward("Key", new Change<>("Key->newValue!", "Key->oldValue!")); +// expectLastCall(); +// replay(context); +// +// processor.process("Key", new Change<>("newValue", "oldValue")); +// +// verify(context); +// } @Test public void shouldNotSetSendOldValuesOnParentIfMaterialized() { @@ -217,12 +217,12 @@ public void shouldTransformOnGetIfNotMaterialized() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); - expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); expect(parentGetterSupplier.get()).andReturn(parentGetter); expect(parentGetter.get("Key")).andReturn(ValueAndTimestamp.make("Value", -1L)); replay(parent, parentGetterSupplier, parentGetter); - final KTableValueGetter getter = transformValues.view().get(); + final KTableValueAndTimestampGetter getter = transformValues.view().get(); getter.init(context); final String result = getter.get("Key").value(); @@ -230,6 +230,7 @@ public void shouldTransformOnGetIfNotMaterialized() { assertThat(result, is("Key->Value!")); } + @SuppressWarnings("unchecked") @Test public void shouldGetFromStateStoreIfMaterialized() { final KTableTransformValues transformValues = @@ -239,7 +240,7 @@ public void shouldGetFromStateStoreIfMaterialized() { expect(stateStore.get("Key")).andReturn(ValueAndTimestamp.make("something", 0L)); replay(context, stateStore); - final KTableValueGetter getter = transformValues.view().get(); + final KTableValueAndTimestampGetter getter = transformValues.view().get(); getter.init(context); final String result = getter.get("Key").value(); @@ -252,7 +253,7 @@ public void shouldGetStoreNamesFromParentIfNotMaterialized() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); - expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); expect(parentGetterSupplier.storeNames()).andReturn(new String[]{"store1", "store2"}); replay(parent, parentGetterSupplier); @@ -271,21 +272,22 @@ public void shouldGetQueryableStoreNameIfMaterialized() { assertThat(storeNames, is(new String[]{QUERYABLE_NAME})); } - @Test - public void shouldCloseTransformerOnProcessorClose() { - final KTableTransformValues transformValues = - new KTableTransformValues<>(parent, mockSupplier, null); - - expect(mockSupplier.get()).andReturn(transformer); - transformer.close(); - expectLastCall(); - replay(mockSupplier, transformer); - - final Processor> processor = transformValues.get(); - processor.close(); - - verify(transformer); - } + // TODO +// @Test +// public void shouldCloseTransformerOnProcessorClose() { +// final KTableTransformValues transformValues = +// new KTableTransformValues<>(parent, mockSupplier, null); +// +// expect(mockSupplier.get()).andReturn(transformer); +// transformer.close(); +// expectLastCall(); +// replay(mockSupplier, transformer); +// +// final Processor> processor = transformValues.get(); +// processor.close(); +// +// verify(transformer); +// } @Test public void shouldCloseTransformerOnGetterClose() { @@ -294,14 +296,14 @@ public void shouldCloseTransformerOnGetterClose() { expect(mockSupplier.get()).andReturn(transformer); expect(parentGetterSupplier.get()).andReturn(parentGetter); - expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); transformer.close(); expectLastCall(); replay(mockSupplier, transformer, parent, parentGetterSupplier); - final KTableValueGetter getter = transformValues.view().get(); + final KTableValueAndTimestampGetter getter = transformValues.view().get(); getter.close(); verify(transformer); @@ -312,7 +314,7 @@ public void shouldCloseParentGetterClose() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, mockSupplier, null); - expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); expect(mockSupplier.get()).andReturn(transformer); expect(parentGetterSupplier.get()).andReturn(parentGetter); @@ -321,7 +323,7 @@ public void shouldCloseParentGetterClose() { replay(mockSupplier, parent, parentGetterSupplier, parentGetter); - final KTableValueGetter getter = transformValues.view().get(); + final KTableValueAndTimestampGetter getter = transformValues.view().get(); getter.close(); verify(parentGetter); @@ -345,7 +347,7 @@ public void shouldTransformValuesWithKey() { inputTopic.pipeInput("A", "a", 5L); inputTopic.pipeInput("B", "b", 10L); - inputTopic.pipeInput("D", (String) null, 15L); + inputTopic.pipeInput("D", null, 15L); assertThat(output(), hasItems(new KeyValueTimestamp<>("A", "A->a!", 5), @@ -374,7 +376,7 @@ public void shouldTransformValuesWithKeyAndMaterialize() { driver.createInputTopic(INPUT_TOPIC, new StringSerializer(), new StringSerializer()); inputTopic.pipeInput("A", "a", 5L); inputTopic.pipeInput("B", "b", 10L); - inputTopic.pipeInput("C", (String) null, 15L); + inputTopic.pipeInput("C", null, 15L); assertThat(output(), hasItems(new KeyValueTimestamp<>("A", "A->a!", 5), new KeyValueTimestamp<>("B", "B->b!", 10), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java index b25febf94072b..38cfacbc906e8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java @@ -28,6 +28,7 @@ import static org.easymock.EasyMock.verify; public class SessionCacheFlushListenerTest { + @SuppressWarnings("unchecked") @Test public void shouldForwardKeyNewValueOldValueAndTimestamp() { final InternalProcessorContext context = mock(InternalProcessorContext.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java index d6e56ba5b6128..c08092b241cc8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java @@ -80,7 +80,7 @@ public void shouldCountSessionWindowedWithCachingEnabled() { } private void shouldCountSessionWindowed() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); stream.count() .toStream() .process(supplier); @@ -106,7 +106,7 @@ private void shouldCountSessionWindowed() { @Test public void shouldReduceWindowed() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); stream.reduce(MockReducer.STRING_ADDER) .toStream() .process(supplier); @@ -132,7 +132,7 @@ public void shouldReduceWindowed() { @Test public void shouldAggregateSessionWindowed() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); stream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, sessionMerger, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java index d6b26bf2a7c59..4abbcd9b54c2e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java @@ -71,7 +71,7 @@ public void before() { @Test public void shouldCountSlidingWindows() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); windowedStream .count() .toStream() @@ -112,7 +112,7 @@ public void shouldCountSlidingWindows() { @Test public void shouldReduceSlidingWindows() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); windowedStream .reduce(MockReducer.STRING_ADDER) .toStream() @@ -153,7 +153,7 @@ public void shouldReduceSlidingWindows() { @Test public void shouldAggregateSlidingWindows() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); windowedStream .aggregate( MockInitializer.STRING_INIT, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java index c35da00697ba5..792c36783e3ae 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java @@ -68,7 +68,7 @@ public void before() { @Test public void shouldCountWindowed() { - final MockProcessorSupplier, Long> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); windowedStream .count() .toStream() @@ -93,7 +93,7 @@ public void shouldCountWindowed() { @Test public void shouldReduceWindowed() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); windowedStream .reduce(MockReducer.STRING_ADDER) .toStream() @@ -118,7 +118,7 @@ public void shouldReduceWindowed() { @Test public void shouldAggregateWindowed() { - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); windowedStream .aggregate( MockInitializer.STRING_INIT, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java index 0d75452a798fc..712ebb77c6556 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java @@ -22,8 +22,9 @@ import org.apache.kafka.streams.kstream.internals.KStreamSessionWindowAggregate; import org.apache.kafka.streams.kstream.internals.KStreamWindowAggregate; import org.apache.kafka.streams.kstream.internals.TimeWindow; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.StoreBuilder; import org.junit.Test; @@ -50,12 +51,13 @@ public void shouldFailIfThereIsNoGraceAncestor() { final StatefulProcessorNode gracelessAncestor = new StatefulProcessorNode<>( "stateful", new ProcessorParameters<>( - () -> new Processor() { + () -> new Processor() { @Override - public void init(final ProcessorContext context) {} + public void init(final ProcessorContext context) {} @Override - public void process(final String key, final Long value) {} + public void process(Record record) { + } @Override public void close() {} @@ -134,12 +136,13 @@ public void shouldExtractGraceFromSessionAncestorThroughStatefulParent() { final StatefulProcessorNode statefulParent = new StatefulProcessorNode<>( "stateful", new ProcessorParameters<>( - () -> new Processor() { + () -> new Processor() { @Override - public void init(final ProcessorContext context) {} + public void init(final ProcessorContext context) {} @Override - public void process(final String key, final Long value) {} + public void process(Record record) { + } @Override public void close() {} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java index 778af9a00dcfc..0c50886bc288e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java @@ -28,8 +28,8 @@ import org.apache.kafka.streams.kstream.internals.KTableImpl; import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.kstream.internals.TimeWindow; -import org.apache.kafka.streams.processor.MockProcessorContext; -import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.api.MockProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.internals.ProcessorNode; @@ -65,7 +65,7 @@ public class KTableSuppressProcessorTest { private static final Change ARBITRARY_CHANGE = new Change<>(7L, 14L); private static class Harness { - private final Processor> processor; + private final Processor, ?, ?> processor; private final MockInternalProcessorContext context; @@ -80,7 +80,7 @@ private static class Harness { .build(); final KTableImpl parent = EasyMock.mock(KTableImpl.class); - final Processor> processor = + final Processor, ?, ?> processor = new KTableSuppressProcessorSupplier<>((SuppressedInternal) suppressed, storeName, parent).get(); final MockInternalProcessorContext context = new MockInternalProcessorContext(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java index 1f4cc824e486b..57d6936d03570 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java @@ -171,7 +171,7 @@ public void appConfigsShouldReturnUnrecognizedValues() { ); } - private static class TestProcessorContext extends AbstractProcessorContext { + private static class TestProcessorContext extends AbstractProcessorContext { static Properties config; static { config = getStreamsConfig(); diff --git a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java index 55f22a2529e8b..3b4fe1e9fd9e8 100644 --- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java @@ -59,8 +59,8 @@ import static org.apache.kafka.streams.processor.internals.StateRestoreCallbackAdapter.adapt; -public class InternalMockProcessorContext - extends AbstractProcessorContext +public class InternalMockProcessorContext + extends AbstractProcessorContext implements RecordCollector.Supplier { private StateManager stateManager = new StateManagerStub(); @@ -293,16 +293,13 @@ public Cancellable schedule(final Duration interval, } @Override - public void commit() {} - - @Override - public void forward(final Record record) { + public void forward(Record record) { forward(record, null); } @SuppressWarnings("unchecked") @Override - public void forward(final Record record, final String childName) { + public void forward(Record record, String childName) { if (recordContext != null && record.timestamp() != recordContext.timestamp()) { setTime(record.timestamp()); } @@ -310,13 +307,16 @@ public void forward(final Record record, final String childName) { try { for (final ProcessorNode childNode : thisNode.children()) { currentNode = childNode; - ((ProcessorNode) childNode).process(record); + ((ProcessorNode) childNode).process(record); } } finally { currentNode = thisNode; } } + @Override + public void commit() {} + @Override public void forward(final Object key, final Object value) { forward(key, value, To.all()); diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java index f18b763f0fe3e..45a1b974495fa 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -17,10 +17,10 @@ package org.apache.kafka.test; import org.apache.kafka.streams.KeyValueTimestamp; -import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Cancellable; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -28,8 +28,8 @@ import java.util.List; import java.util.Map; -public class MockProcessor extends AbstractProcessor { - private final MockApiProcessor delegate; +public class MockProcessor implements Processor { + private final MockApiProcessor delegate; public MockProcessor(final PunctuationType punctuationType, final long scheduleInterval) { @@ -40,16 +40,14 @@ public MockProcessor() { delegate = new MockApiProcessor<>(); } - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - super.init(context); - delegate.init((org.apache.kafka.streams.processor.api.ProcessorContext) context); + public void init(final ProcessorContext context) { + delegate.init(context); } @Override - public void process(final K key, final V value) { - delegate.process(new Record<>(key, value, context.timestamp(), context.headers())); + public void process(final Record record) { + delegate.process(record); } public void checkAndClearProcessResult(final KeyValueTimestamp... expected) { @@ -68,7 +66,7 @@ public void checkAndClearPunctuateResult(final PunctuationType type, final long. delegate.checkAndClearPunctuateResult(type, expected); } - public Map> lastValueAndTimestampPerKey() { + public Map> lastValueAndTimestampPerKey() { return delegate.lastValueAndTimestampPerKey(); } @@ -80,7 +78,7 @@ public Cancellable scheduleCancellable() { return delegate.scheduleCancellable(); } - public ArrayList> processed() { + public ArrayList> processed() { return delegate.processed(); } } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java index 4ab4cb8bbbcd3..a1e274a91d5ee 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java @@ -29,7 +29,7 @@ public class MockProcessorNode extends ProcessorNode mockProcessor; + public final MockProcessor mockProcessor; public boolean closed; public boolean initialized; @@ -46,7 +46,7 @@ public MockProcessorNode() { this(new MockProcessor<>()); } - private MockProcessorNode(final MockProcessor mockProcessor) { + private MockProcessorNode(final MockProcessor mockProcessor) { super(NAME + INDEX.getAndIncrement(), mockProcessor, Collections.emptySet()); this.mockProcessor = mockProcessor; diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java index 7b73dd3d5b7af..b1f9a2899bf52 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java @@ -16,20 +16,21 @@ */ package org.apache.kafka.test; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.PunctuationType; import java.util.ArrayList; import java.util.List; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import static org.junit.Assert.assertEquals; -public class MockProcessorSupplier implements ProcessorSupplier { +public class MockProcessorSupplier implements + ProcessorSupplier { private final long scheduleInterval; private final PunctuationType punctuationType; - private final List> processors = new ArrayList<>(); + private final List> processors = new ArrayList<>(); public MockProcessorSupplier() { this(-1L); @@ -45,8 +46,8 @@ public MockProcessorSupplier(final long scheduleInterval, final PunctuationType } @Override - public Processor get() { - final MockProcessor processor = new MockProcessor<>(punctuationType, scheduleInterval); + public Processor get() { + final MockProcessor processor = new MockProcessor<>(punctuationType, scheduleInterval); // to keep tests simple, ignore calls from ApiUtils.checkSupplier if (!StreamsTestUtils.isCheckSupplierCall()) { @@ -57,7 +58,7 @@ public Processor get() { } // get the captured processor assuming that only one processor gets returned from this supplier - public MockProcessor theCapturedProcessor() { + public MockProcessor theCapturedProcessor() { return capturedProcessors(1).get(0); } @@ -66,7 +67,7 @@ public int capturedProcessorsCount() { } // get the captured processors with the expected number - public List> capturedProcessors(final int expectedNumberOfProcessors) { + public List> capturedProcessors(final int expectedNumberOfProcessors) { assertEquals(expectedNumberOfProcessors, processors.size()); return processors; diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java index b596a7ef82c8c..c92fa62aaebee 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java @@ -44,7 +44,7 @@ import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener; -public class NoOpProcessorContext extends AbstractProcessorContext { +public class NoOpProcessorContext extends AbstractProcessorContext { public boolean initialized; @SuppressWarnings("WeakerAccess") public Map forwardedValues = new HashMap<>(); From 8bd10187ed353872bd5d684b00c763ee62d6b7ad Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 14:19:59 +0100 Subject: [PATCH 17/40] migrate recordqueue --- .../streams/errors/DeserializationExceptionHandler.java | 7 ++++--- .../streams/errors/LogAndContinueExceptionHandler.java | 4 ++-- .../kafka/streams/errors/LogAndFailExceptionHandler.java | 4 ++-- .../streams/processor/internals/RecordDeserializer.java | 2 +- .../kafka/streams/processor/internals/RecordQueue.java | 6 +++--- 5 files changed, 12 insertions(+), 11 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java index 4c382b6932ce6..d931177a98429 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java @@ -19,7 +19,8 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.Configurable; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; /** * Interface that specifies how an exception from source node deserialization @@ -31,14 +32,14 @@ public interface DeserializationExceptionHandler extends Configurable { * Inspect a record and the exception received. *

* Note, that the passed in {@link ProcessorContext} only allows to access metadata like the task ID. - * However, it cannot be used to emit records via {@link ProcessorContext#forward(Object, Object)}; + * However, it cannot be used to emit records via {@link ProcessorContext#forward(Record) ()}; * calling {@code forward()} (and some other methods) would result in a runtime exception. * * @param context processor context * @param record record that failed deserialization * @param exception the actual exception */ - DeserializationHandlerResponse handle(final ProcessorContext context, + DeserializationHandlerResponse handle(final ProcessorContext context, final ConsumerRecord record, final Exception exception); diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java index 4f9a0964405a2..8ac66db59aa17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java @@ -18,7 +18,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,7 +32,7 @@ public class LogAndContinueExceptionHandler implements DeserializationExceptionH private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); @Override - public DeserializationHandlerResponse handle(final ProcessorContext context, + public DeserializationHandlerResponse handle(final ProcessorContext context, final ConsumerRecord record, final Exception exception) { diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java index 61d210649ba9a..3375149a85496 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,7 +32,7 @@ public class LogAndFailExceptionHandler implements DeserializationExceptionHandl private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); @Override - public DeserializationHandlerResponse handle(final ProcessorContext context, + public DeserializationHandlerResponse handle(final ProcessorContext context, final ConsumerRecord record, final Exception exception) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 86b1b442be6c7..998b3f4d81160 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -22,7 +22,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.slf4j.Logger; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 6f0db8a648865..697f03e1a35b6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -23,7 +23,7 @@ import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.TimestampExtractor; import org.slf4j.Logger; @@ -41,7 +41,7 @@ public class RecordQueue { private final Logger log; private final SourceNode source; private final TopicPartition partition; - private final ProcessorContext processorContext; + private final ProcessorContext processorContext; private final TimestampExtractor timestampExtractor; private final RecordDeserializer recordDeserializer; private final ArrayDeque> fifoQueue; @@ -55,7 +55,7 @@ public class RecordQueue { final SourceNode source, final TimestampExtractor timestampExtractor, final DeserializationExceptionHandler deserializationExceptionHandler, - final InternalProcessorContext processorContext, + final InternalProcessorContext processorContext, final LogContext logContext) { this.source = source; this.partition = partition; From 31f51b0902abe014ffe8a246e8e27ffe745ebae1 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 16:42:25 +0100 Subject: [PATCH 18/40] migrate testing --- .../kstream/internals/AbstractStream.java | 80 +++++++++---------- .../kstream/internals/KStreamImpl.java | 50 +++++++----- .../GlobalThreadShutDownOrderTest.java | 18 ++--- .../StoreUpgradeIntegrationTest.java | 69 +++++++--------- ...caughtExceptionHandlerIntegrationTest.java | 10 ++- .../kstream/RepartitionTopicNamingTest.java | 9 ++- .../kstream/internals/AbstractStreamTest.java | 46 +++++------ .../internals/KGroupedStreamImplTest.java | 6 +- .../kstream/internals/KStreamBranchTest.java | 2 +- .../kstream/internals/KStreamFilterTest.java | 4 +- .../kstream/internals/KStreamFlatMapTest.java | 3 +- .../internals/KStreamFlatMapValuesTest.java | 5 +- .../internals/KStreamFlatTransformTest.java | 30 +++---- .../internals/KStreamKStreamJoinTest.java | 12 +-- .../internals/KStreamKTableJoinTest.java | 2 +- .../kstream/internals/KStreamMapTest.java | 2 +- .../internals/KStreamSelectKeyTest.java | 2 +- .../KStreamSlidingWindowAggregateTest.java | 18 ++--- .../internals/KStreamTransformTest.java | 4 +- .../internals/KStreamWindowAggregateTest.java | 4 +- .../kstream/internals/KTableImplTest.java | 4 +- .../kstream/internals/KTableMapKeysTest.java | 2 +- .../SessionWindowedKStreamImplTest.java | 6 +- .../SlidingWindowedKStreamImplTest.java | 6 +- .../TimeWindowedKStreamImplTest.java | 6 +- .../graph/TableProcessorNodeTest.java | 15 +--- .../processor/internals/RecordQueueTest.java | 2 +- .../processor/internals/SinkNodeTest.java | 2 +- .../internals/AbstractKeyValueStoreTest.java | 4 +- .../AbstractSessionBytesStoreTest.java | 6 +- .../AbstractWindowBytesStoreTest.java | 4 +- .../state/internals/RocksDBStoreTest.java | 8 +- .../state/internals/SegmentIteratorTest.java | 2 +- 33 files changed, 216 insertions(+), 227 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java index b14574193c7e5..2fac9ba69ab3b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java @@ -16,25 +16,17 @@ */ package org.apache.kafka.streams.kstream.internals; +import java.util.Collection; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.kstream.ValueTransformer; -import org.apache.kafka.streams.kstream.ValueTransformerSupplier; -import org.apache.kafka.streams.kstream.ValueTransformerWithKey; -import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.graph.GraphNode; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; -import org.apache.kafka.streams.state.StoreBuilder; - -import java.util.Collection; -import java.util.HashSet; -import java.util.Objects; -import java.util.Set; /* * Any classes (KTable, KStream, etc) extending this class should follow the serde specification precedence ordering as: @@ -111,38 +103,38 @@ static ValueMapperWithKey withKey(final ValueMapper return (readOnlyKey, value) -> valueMapper.apply(value); } - static ValueTransformerWithKeySupplier toValueTransformerWithKeySupplier( - final ValueTransformerSupplier valueTransformerSupplier) { - Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); - ApiUtils.checkSupplier(valueTransformerSupplier); - return new ValueTransformerWithKeySupplier() { - @Override - public ValueTransformerWithKey get() { - final ValueTransformer valueTransformer = valueTransformerSupplier.get(); - return new ValueTransformerWithKey() { - @Override - public void init(final ProcessorContext context) { - valueTransformer.init(context); - } - - @Override - public VR transform(final K readOnlyKey, final V value) { - return valueTransformer.transform(value); - } - - @Override - public void close() { - valueTransformer.close(); - } - }; - } - - @Override - public Set> stores() { - return valueTransformerSupplier.stores(); - } - }; - } +// static ValueTransformerWithKeySupplier toValueTransformerWithKeySupplier( +// final ValueTransformerSupplier valueTransformerSupplier) { +// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); +// ApiUtils.checkSupplier(valueTransformerSupplier); +// return new ValueTransformerWithKeySupplier() { +// @Override +// public ValueTransformerWithKey get() { +// final ValueTransformer valueTransformer = valueTransformerSupplier.get(); +// return new ValueTransformerWithKey() { +// @Override +// public void init(final ProcessorContext context) { +// valueTransformer.init(context); +// } +// +// @Override +// public VR transform(final K readOnlyKey, final V value) { +// return valueTransformer.transform(value); +// } +// +// @Override +// public void close() { +// valueTransformer.close(); +// } +// }; +// } +// +// @Override +// public Set> stores() { +// return valueTransformerSupplier.stores(); +// } +// }; +// } static ValueJoinerWithKey toValueJoinerWithKey(final ValueJoiner valueJoiner) { Objects.requireNonNull(valueJoiner, "joiner can't be null"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 161810930ac5a..923992a0849ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1336,23 +1336,27 @@ public KStream flatTransform(final TransformerSupplier KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, final String... stateStoreNames) { - Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); - return doTransformValues( - toValueTransformerWithKeySupplier(valueTransformerSupplier), - NamedInternal.empty(), - stateStoreNames); + //TODO + return null; +// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); +// return doTransformValues( +// toValueTransformerWithKeySupplier(valueTransformerSupplier), +// NamedInternal.empty(), +// stateStoreNames); } @Override public KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, final Named named, final String... stateStoreNames) { - Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); - Objects.requireNonNull(named, "named can't be null"); - return doTransformValues( - toValueTransformerWithKeySupplier(valueTransformerSupplier), - new NamedInternal(named), - stateStoreNames); + //TODO + return null; +// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); +// Objects.requireNonNull(named, "named can't be null"); +// return doTransformValues( +// toValueTransformerWithKeySupplier(valueTransformerSupplier), +// new NamedInternal(named), +// stateStoreNames); } @Override @@ -1403,22 +1407,26 @@ private KStream doTransformValues(final ValueTransformerWithKeySuppl @Override public KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, final String... stateStoreNames) { - Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); - return doFlatTransformValues( - toValueTransformerWithKeySupplier(valueTransformerSupplier), - NamedInternal.empty(), - stateStoreNames); + //TODO + return null; +// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); +// return doFlatTransformValues( +// toValueTransformerWithKeySupplier(valueTransformerSupplier), +// NamedInternal.empty(), +// stateStoreNames); } @Override public KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, final Named named, final String... stateStoreNames) { - Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); - return doFlatTransformValues( - toValueTransformerWithKeySupplier(valueTransformerSupplier), - named, - stateStoreNames); + //TODO + return null; +// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); +// return doFlatTransformValues( +// toValueTransformerWithKeySupplier(valueTransformerSupplier), +// named, +// stateStoreNames); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java index 3bc99ac42c404..4c717e395ac76 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java @@ -30,8 +30,9 @@ import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; @@ -179,7 +180,7 @@ private void createTopics() throws Exception { } - private void populateTopics(final String topicName) throws Exception { + private void populateTopics(final String topicName) { IntegrationTestUtils.produceKeyValuesSynchronously( topicName, Arrays.asList( @@ -196,7 +197,7 @@ private void populateTopics(final String topicName) throws Exception { } - private class GlobalStoreProcessor extends AbstractProcessor { + private class GlobalStoreProcessor implements Processor { private KeyValueStore store; private final String storeName; @@ -206,14 +207,12 @@ private class GlobalStoreProcessor extends AbstractProcessor { } @Override - @SuppressWarnings("unchecked") - public void init(final ProcessorContext context) { - super.init(context); - store = (KeyValueStore) context.getStateStore(storeName); + public void init(final ProcessorContext context) { + store = context.getStateStore(storeName); } @Override - public void process(final String key, final Long value) { + public void process(final Record record) { firstRecordProcessed = true; } @@ -229,5 +228,4 @@ public void close() { } } } - } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java index 12dfdec7662f2..20dc383942757 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java @@ -27,8 +27,9 @@ import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; @@ -953,120 +954,112 @@ private void processKeyValueAndVerifyWindowedCountWithTimestamp(final K k "Could not get expected result in time."); } - private static class KeyValueProcessor implements Processor { + private static class KeyValueProcessor implements Processor { private KeyValueStore store; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - store = (KeyValueStore) context.getStateStore(STORE_NAME); + public void init(final ProcessorContext context) { + store = context.getStateStore(STORE_NAME); } @Override - public void process(final Integer key, final Integer value) { + public void process(final Record record) { final long newCount; - final Long oldCount = store.get(key); + final Long oldCount = store.get(record.key()); if (oldCount != null) { newCount = oldCount + 1L; } else { newCount = 1L; } - store.put(key, newCount); + store.put(record.key(), newCount); } @Override public void close() {} } - private static class TimestampedKeyValueProcessor implements Processor { - private ProcessorContext context; + private static class TimestampedKeyValueProcessor implements Processor { private TimestampedKeyValueStore store; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - this.context = context; - store = (TimestampedKeyValueStore) context.getStateStore(STORE_NAME); + public void init(final ProcessorContext context) { + store = context.getStateStore(STORE_NAME); } @Override - public void process(final Integer key, final Integer value) { + public void process(final Record record) { final long newCount; - final ValueAndTimestamp oldCountWithTimestamp = store.get(key); + final ValueAndTimestamp oldCountWithTimestamp = store.get(record.key()); final long newTimestamp; if (oldCountWithTimestamp == null) { newCount = 1L; - newTimestamp = context.timestamp(); + newTimestamp = record.timestamp(); } else { newCount = oldCountWithTimestamp.value() + 1L; - newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), context.timestamp()); + newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), record.timestamp()); } - store.put(key, ValueAndTimestamp.make(newCount, newTimestamp)); + store.put(record.key(), ValueAndTimestamp.make(newCount, newTimestamp)); } @Override public void close() {} } - private static class WindowedProcessor implements Processor { + private static class WindowedProcessor implements Processor { private WindowStore store; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - store = (WindowStore) context.getStateStore(STORE_NAME); + public void init(final ProcessorContext context) { + store = context.getStateStore(STORE_NAME); } @Override - public void process(final Integer key, final Integer value) { + public void process(final Record record) { final long newCount; - final Long oldCount = store.fetch(key, key < 10 ? 0L : 100000L); + final Long oldCount = store.fetch(record.key(), record.key() < 10 ? 0L : 100000L); if (oldCount != null) { newCount = oldCount + 1L; } else { newCount = 1L; } - store.put(key, newCount, key < 10 ? 0L : 100000L); + store.put(record.key(), newCount, record.key() < 10 ? 0L : 100000L); } @Override public void close() {} } - private static class TimestampedWindowedProcessor implements Processor { - private ProcessorContext context; + private static class TimestampedWindowedProcessor implements Processor { private TimestampedWindowStore store; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { - this.context = context; - store = (TimestampedWindowStore) context.getStateStore(STORE_NAME); + public void init(final ProcessorContext context) { + store = context.getStateStore(STORE_NAME); } @Override - public void process(final Integer key, final Integer value) { + public void process(final Record record) { final long newCount; - final ValueAndTimestamp oldCountWithTimestamp = store.fetch(key, key < 10 ? 0L : 100000L); + final ValueAndTimestamp oldCountWithTimestamp = store.fetch(record.key(), record.key() < 10 ? 0L : 100000L); final long newTimestamp; if (oldCountWithTimestamp == null) { newCount = 1L; - newTimestamp = context.timestamp(); + newTimestamp = record.timestamp(); } else { newCount = oldCountWithTimestamp.value() + 1L; - newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), context.timestamp()); + newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), record.timestamp()); } - store.put(key, ValueAndTimestamp.make(newCount, newTimestamp), key < 10 ? 0L : 100000L); + store.put(record.key(), ValueAndTimestamp.make(newCount, newTimestamp), record.key() < 10 ? 0L : 100000L); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index cc1b445525f62..f037a230f64a5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -31,6 +31,9 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Named; import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.test.IntegrationTest; @@ -209,7 +212,6 @@ public void shouldShutDownClientIfGlobalStreamThreadWantsToReplaceThread() throw assertThat(processorValueCollector.size(), equalTo(1)); } - } private void produceMessages(final long timestamp, final String streamOneInput, final String msg) { @@ -224,7 +226,7 @@ private void produceMessages(final long timestamp, final String streamOneInput, timestamp); } - private static class ShutdownProcessor extends AbstractProcessor { + private static class ShutdownProcessor extends ContextualProcessor { final List valueList; ShutdownProcessor(final List valueList) { @@ -232,8 +234,8 @@ private static class ShutdownProcessor extends AbstractProcessor } @Override - public void process(final String key, final String value) { - valueList.add(value + " " + context.taskId()); + public void process(Record record) { + valueList.add(record + " " + context.taskId()); if (throwError.get()) { throw new StreamsException(Thread.currentThread().getName()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java index 3091a6f9e24c8..c1ed533271715 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java @@ -23,7 +23,8 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.errors.TopologyException; -import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.junit.Test; import java.time.Duration; @@ -505,7 +506,7 @@ private Topology buildTopology(final String optimizationConfig) { } - private static class SimpleProcessor extends AbstractProcessor { + private static class SimpleProcessor implements Processor { final List valueList; @@ -514,8 +515,8 @@ private static class SimpleProcessor extends AbstractProcessor { } @Override - public void process(final String key, final String value) { - valueList.add(value); + public void process(Record record) { + valueList.add(record.value()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java index cfe50e266241a..8b78ca7e91c6b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java @@ -19,20 +19,19 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.test.NoopValueTransformer; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.test.NoopValueTransformerWithKey; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.ValueTransformerSupplier; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode; import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.test.MockProcessorSupplier; import org.junit.Test; @@ -46,18 +45,19 @@ public class AbstractStreamTest { - @Test - public void testToInternalValueTransformerSupplierSuppliesNewTransformers() { - final ValueTransformerSupplier valueTransformerSupplier = createMock(ValueTransformerSupplier.class); - expect(valueTransformerSupplier.get()).andAnswer(NoopValueTransformer::new).atLeastOnce(); - replay(valueTransformerSupplier); - final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier = - AbstractStream.toValueTransformerWithKeySupplier(valueTransformerSupplier); - valueTransformerWithKeySupplier.get(); - valueTransformerWithKeySupplier.get(); - valueTransformerWithKeySupplier.get(); - verify(valueTransformerSupplier); - } + //TODO +// @Test +// public void testToInternalValueTransformerSupplierSuppliesNewTransformers() { +// final ValueTransformerSupplier valueTransformerSupplier = createMock(ValueTransformerSupplier.class); +// expect(valueTransformerSupplier.get()).andAnswer(NoopValueTransformer::new).atLeastOnce(); +// replay(valueTransformerSupplier); +// final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier = +// AbstractStream.toValueTransformerWithKeySupplier(valueTransformerSupplier); +// valueTransformerWithKeySupplier.get(); +// valueTransformerWithKeySupplier.get(); +// valueTransformerWithKeySupplier.get(); +// verify(valueTransformerSupplier); +// } @Test public void testToInternalValueTransformerWithKeySupplierSuppliesNewTransformers() { @@ -75,7 +75,7 @@ public void testToInternalValueTransformerWithKeySupplierSuppliesNewTransformers public void testShouldBeExtensible() { final StreamsBuilder builder = new StreamsBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final String topicName = "topic"; final ExtendedKStream stream = new ExtendedKStream<>(builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()))); @@ -108,7 +108,7 @@ KStream randomFilter() { } } - private static class ExtendedKStreamDummy implements ProcessorSupplier { + private static class ExtendedKStreamDummy implements ProcessorSupplier { private final Random rand; @@ -117,16 +117,16 @@ private static class ExtendedKStreamDummy implements ProcessorSupplier get() { + public Processor get() { return new ExtendedKStreamDummyProcessor(); } - private class ExtendedKStreamDummyProcessor extends AbstractProcessor { + private class ExtendedKStreamDummyProcessor extends ContextualProcessor { @Override - public void process(final K key, final V value) { + public void process(Record record) { // flip a coin and filter if (rand.nextBoolean()) { - context().forward(key, value); + context().forward(record); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index b733959729789..562ea891f28e2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -202,7 +202,7 @@ public void shouldNotHaveInvalidStoreNameOnSlidingWindowedAggregate() { @Test public void shouldCountSlidingWindows() { - final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(500L), ofMillis(2000L))) .count(Materialized.as("aggregate-by-key-windowed")) @@ -214,7 +214,7 @@ public void shouldCountSlidingWindows() { @Test public void shouldCountSlidingWindowsWithInternalStoreName() { - final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(500L), ofMillis(2000L))) .count() @@ -792,7 +792,7 @@ private void doCountWindowed(final MockProcessorSupplier, Long @Test public void shouldCountWindowed() { - final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(TimeWindows.of(ofMillis(500L))) .count(Materialized.as("aggregate-by-key-windowed")) diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index c947338b2d6c0..3531db78c43d2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -59,7 +59,7 @@ public void testKStreamBranch() { assertEquals(3, branches.length); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); for (final KStream branch : branches) { branch.process(supplier); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index 4ecc24079ccad..ed7297df2b865 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -46,7 +46,7 @@ public void testFilter() { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; final KStream stream; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.filter(isMultipleOfThree).process(supplier); @@ -67,7 +67,7 @@ public void testFilterNot() { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; final KStream stream; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.filterNot(isMultipleOfThree).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index 2597a663ea3d7..431de64bae1f2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import java.util.List; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -58,7 +59,7 @@ public void testFlatMap() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream; - final MockProcessorSupplier supplier; + final MockProcessorSupplier> supplier; supplier = new MockProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index b2ae36509b41b..b27d293cb3247 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import java.util.List; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValueTimestamp; @@ -56,7 +57,7 @@ public void testFlatMapValues() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier> supplier = new MockProcessorSupplier<>(); stream.flatMapValues(mapper).process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -92,7 +93,7 @@ public void testFlatMapValuesWithKeys() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier> supplier = new MockProcessorSupplier<>(); stream.flatMapValues(mapper).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java index d18a7a87578da..5dc382cea6817 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java @@ -16,29 +16,29 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertTrue; + +import java.util.Arrays; +import java.util.Collections; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.kstream.internals.KStreamFlatTransform.KStreamFlatTransformProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.junit.Before; import org.junit.Test; -import java.util.Arrays; -import java.util.Collections; - -import static org.junit.Assert.assertTrue; - public class KStreamFlatTransformTest extends EasyMockSupport { private Number inputKey; private Number inputValue; private Transformer>> transformer; - private ProcessorContext context; + private ProcessorContext context; private KStreamFlatTransformProcessor processor; @@ -53,7 +53,7 @@ public void setUp() { @Test public void shouldInitialiseFlatTransformProcessor() { - transformer.init(context); +// TODO transformer.init(context); replayAll(); processor.init(context); @@ -72,11 +72,11 @@ public void shouldTransformInputRecordToMultipleOutputRecords() { EasyMock.expect(transformer.transform(inputKey, inputValue)).andReturn(outputRecords); for (final KeyValue outputRecord : outputRecords) { - context.forward(outputRecord.key, outputRecord.value); + context.forward(new Record<>(outputRecord.key, outputRecord.value, 0L)); } replayAll(); - processor.process(inputKey, inputValue); + processor.process(new Record<>(inputKey, inputValue, 0L)); verifyAll(); } @@ -87,10 +87,10 @@ public void shouldAllowEmptyListAsResultOfTransform() { EasyMock.reset(transformer); EasyMock.expect(transformer.transform(inputKey, inputValue)) - .andReturn(Collections.>emptyList()); + .andReturn(Collections.emptyList()); replayAll(); - processor.process(inputKey, inputValue); + processor.process(new Record<>(inputKey, inputValue, 0L)); verifyAll(); } @@ -104,7 +104,7 @@ public void shouldAllowNullAsResultOfTransform() { .andReturn(null); replayAll(); - processor.process(inputKey, inputValue); + processor.process(new Record<>(inputKey, inputValue, 0L)); verifyAll(); } @@ -129,7 +129,7 @@ public void shouldGetFlatTransformProcessor() { EasyMock.expect(transformerSupplier.get()).andReturn(transformer); replayAll(); - final Processor processor = processorSupplier.get(); + final Processor processor = processorSupplier.get(); verifyAll(); assertTrue(processor instanceof KStreamFlatTransformProcessor); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index e94d49948e9e7..0d78a5f9d628b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -346,7 +346,7 @@ private void runJoin(final StreamJoined streamJoined, final KStream left = builder.stream("left", Consumed.with(Serdes.String(), Serdes.Integer())); final KStream right = builder.stream("right", Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final KStream joinedStream; joinedStream = left.join( @@ -384,7 +384,7 @@ public void testJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); joined = stream1.join( @@ -495,7 +495,7 @@ public void testOuterJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -609,7 +609,7 @@ public void testWindowing() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -1137,7 +1137,7 @@ public void testAsymmetricWindowingAfter() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -1388,7 +1388,7 @@ public void testAsymmetricWindowingBefore() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java index e821d97b0ab71..c97c0be21f47c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java @@ -66,7 +66,7 @@ public class KStreamKTableJoinTest { private TopologyTestDriver driver; private StreamsBuilder builder; private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); @Before public void setUp() { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 94b8066feeeb7..5510af17a7ead 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -45,7 +45,7 @@ public void testMap() { final String topicName = "topic"; final int[] expectedKeys = new int[] {0, 1, 2, 3}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.map((key, value) -> KeyValue.pair(value, key)).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java index 66d84e9b3c85a..9d639a0b00d91 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java @@ -57,7 +57,7 @@ public void testSelectKey() { final KStream stream = builder.stream(topicName, Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); stream.selectKey((key, value) -> keyMap.get(value)).process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java index 3880a545d25dc..68740cb83c956 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java @@ -117,7 +117,7 @@ public void testAggregateSmallInput() { MockAggregator.TOSTRING_ADDER, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = @@ -173,7 +173,7 @@ public void testReduceSmallInput() { MockReducer.STRING_ADDER, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = @@ -231,7 +231,7 @@ public void testAggregateLargeInput() { Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -403,7 +403,7 @@ public void testJoin() { Materialized.as(storeSupplier2) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); table2.toStream().process(supplier); @@ -419,7 +419,7 @@ public void testJoin() { inputTopic1.pipeInput("B", "2", 11L); inputTopic1.pipeInput("C", "3", 12L); - final List, String>> processors = supplier.capturedProcessors(3); + final List, String, Windowed, String>> processors = supplier.capturedProcessors(3); processors.get(0).checkAndClearProcessResult( // left windows created by the first set of records to table 1 @@ -504,7 +504,7 @@ public void testEarlyRecordsSmallInput() { MockAggregator.TOSTRING_ADDER, Materialized.>as("topic-Canonized").withValueSerde(Serdes.String()) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -555,7 +555,7 @@ public void testEarlyRecordsRepeatedInput() { MockAggregator.TOSTRING_ADDER, Materialized.>as("topic-Canonized").withValueSerde(Serdes.String()) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -607,7 +607,7 @@ public void testEarlyRecordsLargeInput() { Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -812,7 +812,7 @@ public void testAggregateRandomInput() { }, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table.toStream().process(supplier); final long seed = new Random().nextLong(); final Random shuffle = new Random(seed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index b821ea66105f9..52087145e7dbd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -73,7 +73,7 @@ public void close() { } final int[] expectedKeys = {1, 10, 100, 1000}; - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); final KStream stream = builder.stream(TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer())); stream.transform(transformerSupplier).process(processor); @@ -134,7 +134,7 @@ public void close() { } final int[] expectedKeys = {1, 10, 100, 1000}; - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); final KStream stream = builder.stream(TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer())); stream.transform(transformerSupplier).process(processor); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 71ca7cc590f98..8b771d15603e0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -79,7 +79,7 @@ public void testAggBasic() { .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -157,7 +157,7 @@ public void testJoin() { .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); final KTable, String> table2 = builder diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 4f4045df9f1f2..6649d73784d95 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -91,7 +91,7 @@ public void testKTable() { final KTable table1 = builder.table(topic1, consumed); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); @@ -159,7 +159,7 @@ public void testMaterializedKTable() { final KTable table1 = builder.table(topic1, consumed, Materialized.as("fred")); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); table1.toStream().process(supplier); final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java index d7500d6171a11..0542af6977b77 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java @@ -59,7 +59,7 @@ public void testMapKeysConvertingToStream() { final int[] originalKeys = new int[] {1, 2, 3}; final String[] values = new String[] {"V_ONE", "V_TWO", "V_THREE"}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); convertedStream.process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java index c08092b241cc8..c4b3d60581148 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java @@ -80,7 +80,7 @@ public void shouldCountSessionWindowedWithCachingEnabled() { } private void shouldCountSessionWindowed() { - final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); stream.count() .toStream() .process(supplier); @@ -106,7 +106,7 @@ private void shouldCountSessionWindowed() { @Test public void shouldReduceWindowed() { - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); stream.reduce(MockReducer.STRING_ADDER) .toStream() .process(supplier); @@ -132,7 +132,7 @@ public void shouldReduceWindowed() { @Test public void shouldAggregateSessionWindowed() { - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); stream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, sessionMerger, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java index 4abbcd9b54c2e..4497b8c902661 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java @@ -71,7 +71,7 @@ public void before() { @Test public void shouldCountSlidingWindows() { - final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); windowedStream .count() .toStream() @@ -112,7 +112,7 @@ public void shouldCountSlidingWindows() { @Test public void shouldReduceSlidingWindows() { - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); windowedStream .reduce(MockReducer.STRING_ADDER) .toStream() @@ -153,7 +153,7 @@ public void shouldReduceSlidingWindows() { @Test public void shouldAggregateSlidingWindows() { - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); windowedStream .aggregate( MockInitializer.STRING_INIT, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java index 792c36783e3ae..dcd31d288ae4f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java @@ -68,7 +68,7 @@ public void before() { @Test public void shouldCountWindowed() { - final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); windowedStream .count() .toStream() @@ -93,7 +93,7 @@ public void shouldCountWindowed() { @Test public void shouldReduceWindowed() { - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); windowedStream .reduce(MockReducer.STRING_ADDER) .toStream() @@ -118,7 +118,7 @@ public void shouldReduceWindowed() { @Test public void shouldAggregateWindowed() { - final MockProcessorSupplier, String, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); windowedStream .aggregate( MockInitializer.STRING_INIT, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java index a2c4938336376..3488a96fa4031 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java @@ -17,24 +17,17 @@ package org.apache.kafka.streams.kstream.internals.graph; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.junit.Test; import static org.junit.Assert.assertTrue; public class TableProcessorNodeTest { - private static class TestProcessor extends AbstractProcessor { - @Override - public void init(final ProcessorContext context) { - } - - @Override - public void process(final String key, final String value) { - } + private static class TestProcessor implements Processor { @Override - public void close() { + public void process(Record record) { } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java index 6929bb8e4c116..f409434b6080b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -60,7 +60,7 @@ public class RecordQueueTest { private final Deserializer intDeserializer = new IntegerDeserializer(); private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class), new MockRecordCollector() ); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java index 30c7b1b3ee337..8d937ef90dca9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java @@ -33,7 +33,7 @@ public class SinkNodeTest { private final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class); private final Serializer anySerializer = Serdes.ByteArray().serializer(); private final RecordCollector recordCollector = new MockRecordCollector(); - private final InternalMockProcessorContext context = new InternalMockProcessorContext(anyStateSerde, recordCollector); + private final InternalMockProcessorContext context = new InternalMockProcessorContext<>(anyStateSerde, recordCollector); private final SinkNode sink = new SinkNode<>("anyNodeName", new StaticTopicNameExtractor<>("any-output-topic"), anySerializer, anySerializer, null); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index 5959a8079106d..f6bf7c1db4f43 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -53,14 +53,14 @@ public abstract class AbstractKeyValueStoreTest { protected abstract KeyValueStore createKeyValueStore(final StateStoreContext context); - protected InternalMockProcessorContext context; + protected InternalMockProcessorContext context; protected KeyValueStore store; protected KeyValueStoreTestDriver driver; @Before public void before() { driver = KeyValueStoreTestDriver.create(Integer.class, String.class); - context = (InternalMockProcessorContext) driver.context(); + context = (InternalMockProcessorContext) driver.context(); context.setTime(10); store = createKeyValueStore(context); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java index 7b63182251ff2..c3d63124a2ead 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java @@ -77,7 +77,7 @@ public abstract class AbstractSessionBytesStoreTest { private MockRecordCollector recordCollector; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; abstract SessionStore buildSessionStore(final long retentionPeriod, final Serde keySerde, @@ -89,7 +89,7 @@ abstract SessionStore buildSessionStore(final long retentionPeriod, public void setUp() { sessionStore = buildSessionStore(RETENTION_PERIOD, Serdes.String(), Serdes.Long()); recordCollector = new MockRecordCollector(); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), @@ -608,7 +608,7 @@ private void shouldLogAndMeasureExpiredRecords(final String builtInMetricsVersio final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); streamsConfig.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); final SessionStore sessionStore = buildSessionStore(RETENTION_PERIOD, Serdes.String(), Serdes.Long()); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), new StreamsConfig(streamsConfig), recordCollector diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 8d73a2d24e36d..57ab7b9d17e53 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -80,7 +80,7 @@ public abstract class AbstractWindowBytesStoreTest { static final long RETENTION_PERIOD = 2 * SEGMENT_INTERVAL; WindowStore windowStore; - InternalMockProcessorContext context; + InternalMockProcessorContext context; MockRecordCollector recordCollector; final File baseDir = TestUtils.tempDirectory("test"); @@ -101,7 +101,7 @@ public void setup() { windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, false, Serdes.Integer(), Serdes.String()); recordCollector = new MockRecordCollector(); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( baseDir, Serdes.String(), Serdes.Integer(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index 0568df2d5689c..faf659bf231d6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -98,7 +98,7 @@ public class RocksDBStoreTest { private final RocksDBMetricsRecorder metricsRecorder = mock(RocksDBMetricsRecorder.class); - InternalMockProcessorContext context; + InternalMockProcessorContext context; RocksDBStore rocksDBStore; @Before @@ -106,7 +106,7 @@ public void setUp() { final Properties props = StreamsTestUtils.getStreamsConfig(); props.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, MockRocksDbConfigSetter.class); dir = TestUtils.tempDirectory(); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( dir, Serdes.String(), Serdes.String(), @@ -304,7 +304,7 @@ public void shouldCallRocksDbConfigSetter() { props.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, MockRocksDbConfigSetter.class); final Object param = new Object(); props.put("abc.def", param); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( dir, Serdes.String(), Serdes.String(), @@ -683,7 +683,7 @@ public void shouldHandleToggleOfEnablingBloomFilters() { final Properties props = StreamsTestUtils.getStreamsConfig(); props.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, TestingBloomFilterRocksDBConfigSetter.class); dir = TestUtils.tempDirectory(); - context = new InternalMockProcessorContext(dir, + context = new InternalMockProcessorContext<>(dir, Serdes.String(), Serdes.String(), new StreamsConfig(props)); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index 97593e71c34f3..ec4aa162e9906 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -55,7 +55,7 @@ public class SegmentIteratorTest { @Before public void before() { - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), From c41ecc78d858e3fec03b3e224e2ffc7bc62bcba8 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 18:19:38 +0100 Subject: [PATCH 19/40] tests compile! --- .../internals/InternalTopologyBuilder.java | 8 +- .../kafka/streams/StreamsBuilderTest.java | 12 +- .../integration/RestoreIntegrationTest.java | 18 +-- .../internals/KGroupedStreamImplTest.java | 2 +- .../KStreamFlatTransformValuesTest.java | 17 +-- ...amSessionWindowAggregateProcessorTest.java | 21 ++-- .../internals/KTableKTableInnerJoinTest.java | 21 ++-- .../internals/KTableKTableOuterJoinTest.java | 23 ++-- .../internals/KTableKTableRightJoinTest.java | 21 ++-- .../kstream/internals/KTableReduceTest.java | 11 +- ...tionResolverJoinProcessorSupplierTest.java | 99 ++++++++------- .../KTableSuppressProcessorMetricsTest.java | 23 ++-- .../suppress/KTableSuppressProcessorTest.java | 119 ++++++++++-------- .../internals/GlobalStreamThreadTest.java | 2 +- .../internals/ProcessorNodeTest.java | 32 +++-- .../internals/RepartitionOptimizingTest.java | 9 +- .../processor/internals/SinkNodeTest.java | 4 +- .../processor/internals/SourceNodeTest.java | 8 +- .../processor/internals/StreamTaskTest.java | 6 +- .../processor/internals/StreamThreadTest.java | 26 ++-- .../state/KeyValueStoreTestDriver.java | 9 +- ...bstractRocksDBSegmentedBytesStoreTest.java | 6 +- .../CachingInMemoryKeyValueStoreTest.java | 6 +- .../CachingInMemorySessionStoreTest.java | 9 +- .../CachingPersistentSessionStoreTest.java | 8 +- .../CachingPersistentWindowStoreTest.java | 6 +- .../ChangeLoggingKeyValueBytesStoreTest.java | 10 +- ...gingTimestampedKeyValueBytesStoreTest.java | 10 +- .../CompositeReadOnlyKeyValueStoreTest.java | 3 +- .../state/internals/KeyValueSegmentsTest.java | 4 +- .../MeteredTimestampedWindowStoreTest.java | 4 +- .../internals/MeteredWindowStoreTest.java | 11 +- .../state/internals/RocksDBStoreTest.java | 10 +- .../internals/TimestampedSegmentsTest.java | 4 +- .../kafka/streams/tests/SmokeTestUtil.java | 101 +++++++-------- .../test/MockInternalProcessorContext.java | 7 +- 36 files changed, 359 insertions(+), 331 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index c8b77a23a2946..78cb1dcd65f1b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -529,14 +529,14 @@ public final void addStateStore(final StoreBuilder storeBuilder, nodeGroups = null; } - public final void addGlobalStore(final StoreBuilder storeBuilder, + public final void addGlobalStore(final StoreBuilder storeBuilder, final String sourceName, final TimestampExtractor timestampExtractor, final Deserializer keyDeserializer, final Deserializer valueDeserializer, final String topic, final String processorName, - final ProcessorSupplier stateUpdateSupplier) { + final ProcessorSupplier stateUpdateSupplier) { Objects.requireNonNull(storeBuilder, "store builder must not be null"); ApiUtils.checkSupplier(stateUpdateSupplier); validateGlobalStoreArguments(sourceName, @@ -550,7 +550,7 @@ public final void addGlobalStore(final StoreBuilder storeBuilder, final String[] topics = {topic}; final String[] predecessors = {sourceName}; - final ProcessorNodeFactory nodeFactory = new ProcessorNodeFactory<>( + final ProcessorNodeFactory nodeFactory = new ProcessorNodeFactory<>( processorName, predecessors, stateUpdateSupplier @@ -675,7 +675,7 @@ public void validateCopartition() { private void validateGlobalStoreArguments(final String sourceName, final String topic, final String processorName, - final ProcessorSupplier stateUpdateSupplier, + final ProcessorSupplier stateUpdateSupplier, final String storeName, final boolean loggingEnabled) { Objects.requireNonNull(sourceName, "sourceName must not be null"); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java index d8f3bf4f650ea..a8874359d7007 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -296,7 +296,7 @@ public void shouldProcessingFromSinkTopic() { final KStream source = builder.stream("topic-source"); source.to("topic-sink"); - final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); source.process(processorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -316,10 +316,10 @@ public void shouldProcessViaThroughTopic() { final KStream source = builder.stream("topic-source"); final KStream through = source.through("topic-sink"); - final MockProcessorSupplier sourceProcessorSupplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier sourceProcessorSupplier = new MockProcessorSupplier<>(); source.process(sourceProcessorSupplier); - final MockProcessorSupplier throughProcessorSupplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier throughProcessorSupplier = new MockProcessorSupplier<>(); through.process(throughProcessorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -337,10 +337,10 @@ public void shouldProcessViaRepartitionTopic() { final KStream source = builder.stream("topic-source"); final KStream through = source.repartition(); - final MockProcessorSupplier sourceProcessorSupplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier sourceProcessorSupplier = new MockProcessorSupplier<>(); source.process(sourceProcessorSupplier); - final MockProcessorSupplier throughProcessorSupplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier throughProcessorSupplier = new MockProcessorSupplier<>(); through.process(throughProcessorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -362,7 +362,7 @@ public void shouldMergeStreams() { final KStream source2 = builder.stream(topic2); final KStream merged = source1.merge(source2); - final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); merged.process(processorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 90890686af4c0..299a75c08443c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -40,10 +40,11 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -430,7 +431,7 @@ static int numStoresClosed() { } } - public static class KeyValueStoreProcessor implements Processor { + public static class KeyValueStoreProcessor implements Processor { private final String topic; private final CountDownLatch processorLatch; @@ -442,16 +443,15 @@ public static class KeyValueStoreProcessor implements Processor) context.getStateStore(topic); + public void init(final ProcessorContext context) { + this.store = context.getStateStore(topic); } @Override - public void process(final Integer key, final Integer value) { - if (key != null) { - store.put(key, value); + public void process(final Record record) { + if (record.key() != null) { + store.put(record.key(), record.value()); processorLatch.countDown(); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 562ea891f28e2..05f5b6d75d726 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -804,7 +804,7 @@ public void shouldCountWindowed() { @Test public void shouldCountWindowedWithInternalStoreName() { - final MockProcessorSupplier, Long, ?, ?> supplier = new MockProcessorSupplier<>(); + final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); groupedStream .windowedBy(TimeWindows.of(ofMillis(500L))) .count() diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java index 36167c04c0ce3..957f2c11a4aab 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java @@ -24,8 +24,9 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.KStreamFlatTransformValues.KStreamFlatTransformValuesProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; @@ -38,7 +39,7 @@ public class KStreamFlatTransformValuesTest extends EasyMockSupport { private Integer inputValue; private ValueTransformerWithKey> valueTransformer; - private ProcessorContext context; + private ProcessorContext context; private KStreamFlatTransformValuesProcessor processor; @@ -72,11 +73,11 @@ public void shouldTransformInputRecordToMultipleOutputValues() { EasyMock.expect(valueTransformer.transform(inputKey, inputValue)).andReturn(outputValues); for (final String outputValue : outputValues) { - context.forward(inputKey, outputValue); + context.forward(new Record<>(inputKey, outputValue, 0)); } replayAll(); - processor.process(inputKey, inputValue); + processor.process(new Record<>(inputKey, inputValue, 0L)); verifyAll(); } @@ -89,7 +90,7 @@ public void shouldEmitNoRecordIfTransformReturnsEmptyList() { EasyMock.expect(valueTransformer.transform(inputKey, inputValue)).andReturn(Collections.emptyList()); replayAll(); - processor.process(inputKey, inputValue); + processor.process(new Record<>(inputKey, inputValue, 0L)); verifyAll(); } @@ -102,7 +103,7 @@ public void shouldEmitNoRecordIfTransformReturnsNull() { EasyMock.expect(valueTransformer.transform(inputKey, inputValue)).andReturn(null); replayAll(); - processor.process(inputKey, inputValue); + processor.process(new Record<>(inputKey, inputValue, 0L)); verifyAll(); } @@ -127,7 +128,7 @@ public void shouldGetFlatTransformValuesProcessor() { EasyMock.expect(valueTransformerSupplier.get()).andReturn(valueTransformer); replayAll(); - final Processor processor = processorSupplier.get(); + final Processor processor = processorSupplier.get(); verifyAll(); assertTrue(processor instanceof KStreamFlatTransformValuesProcessor); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index c2c38e17fcca2..632796f0d01d2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -88,9 +88,9 @@ public class KStreamSessionWindowAggregateProcessorTest { sessionMerger); private final List, Change>> results = new ArrayList<>(); - private final Processor processor = sessionAggregator.get(); + private final Processor, Change> processor = sessionAggregator.get(); private SessionStore sessionStore; - private InternalMockProcessorContext context; + private InternalMockProcessorContext, Change> context; private Metrics metrics; @Before @@ -99,7 +99,7 @@ public void initializeStore() { metrics = new Metrics(); final MockStreamsMetrics metrics = new MockStreamsMetrics(KStreamSessionWindowAggregateProcessorTest.this.metrics); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext, Change>( stateDir, Serdes.String(), Serdes.String(), @@ -109,7 +109,6 @@ public void initializeStore() { new ThreadCache(new LogContext("testCache "), 100000, metrics), Time.SYSTEM ) { - @SuppressWarnings("unchecked") @Override public void forward(final Object key, final Object value, final To to) { toInternal.update(to); @@ -368,7 +367,7 @@ public void shouldLogAndMeterWhenSkippingNullKeyWithBuiltInMetricsVersionLatest( } private void shouldLogAndMeterWhenSkippingNullKeyWithBuiltInMetrics(final String builtInMetricsVersion) { - final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); + final InternalMockProcessorContext, Change> context = createInternalMockProcessorContext(builtInMetricsVersion); processor.init(context); context.setRecordContext( new ProcessorRecordContext(-1, -2, -3, "topic", null) @@ -409,8 +408,8 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithZeroGraceWithBuiltInMetri } private void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace(final String builtInMetricsVersion) { - final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); - final Processor processor = new KStreamSessionWindowAggregate<>( + final InternalMockProcessorContext, Change> context = createInternalMockProcessorContext(builtInMetricsVersion); + final Processor, Change> processor = new KStreamSessionWindowAggregate<>( SessionWindows.with(ofMillis(10L)).grace(ofMillis(0L)), STORE_NAME, initializer, @@ -507,8 +506,8 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGraceWithBuiltInMe } private void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace(final String builtInMetricsVersion) { - final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); - final Processor processor = new KStreamSessionWindowAggregate<>( + final InternalMockProcessorContext, Change> context = createInternalMockProcessorContext(builtInMetricsVersion); + final Processor, Change> processor = new KStreamSessionWindowAggregate<>( SessionWindows.with(ofMillis(10L)).grace(ofMillis(1L)), STORE_NAME, initializer, @@ -602,9 +601,9 @@ private void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace(final Strin greaterThan(0.0)); } - private InternalMockProcessorContext createInternalMockProcessorContext(final String builtInMetricsVersion) { + private InternalMockProcessorContext, Change> createInternalMockProcessorContext(final String builtInMetricsVersion) { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", builtInMetricsVersion, new MockTime()); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext, Change> context = new InternalMockProcessorContext, Change>( TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java index f003b52164521..a77ff5269f0da 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java @@ -28,8 +28,9 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.processor.MockProcessorContext; -import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.api.MockProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.test.TestRecord; @@ -257,23 +258,25 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKeyWithBuiltInMetricsVer shouldLogAndMeterSkippedRecordsDueToNullLeftKey(StreamsConfig.METRICS_0100_TO_24); } + @SuppressWarnings("unchecked") private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtInMetricsVersion) { final StreamsBuilder builder = new StreamsBuilder(); - @SuppressWarnings("unchecked") - final Processor> join = new KTableKTableInnerJoin<>( - (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())), - (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())), + final Processor, String, Change> join = new KTableKTableInnerJoin<>( + (KTableImpl) builder + .table("left", Consumed.with(Serdes.String(), Serdes.String())), + (KTableImpl) builder + .table("right", Consumed.with(Serdes.String(), Serdes.String())), null ).get(); props.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); - final MockProcessorContext context = new MockProcessorContext(props); - context.setRecordMetadata("left", -1, -2, null, -3); + final MockProcessorContext> context = new MockProcessorContext<>(props); + context.setRecordMetadata("left", -1, -2); join.init(context); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KTableKTableInnerJoin.class)) { - join.process(null, new Change<>("new", "old")); + join.process(new Record<>(null, new Change<>("new", "old"), -3)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 9f99ae6028ea1..6c07db130cbf7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -27,8 +27,9 @@ import org.apache.kafka.streams.TopologyWrapper; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.processor.MockProcessorContext; -import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.api.MockProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.test.TestRecord; import org.apache.kafka.test.MockApiProcessor; @@ -414,23 +415,25 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKeyWithBuiltInMetricsVer shouldLogAndMeterSkippedRecordsDueToNullLeftKey(StreamsConfig.METRICS_0100_TO_24); } + @SuppressWarnings("unchecked") private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtInMetricsVersion) { final StreamsBuilder builder = new StreamsBuilder(); - @SuppressWarnings("unchecked") - final Processor> join = new KTableKTableOuterJoin<>( - (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())), - (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())), - null + Processor, String, Change> join = new KTableKTableOuterJoin<>( + (KTableImpl) builder + .table("left", Consumed.with(Serdes.String(), Serdes.String())), + (KTableImpl) builder + .table("right", Consumed.with(Serdes.String(), Serdes.String())), + null ).get(); props.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); - final MockProcessorContext context = new MockProcessorContext(props); - context.setRecordMetadata("left", -1, -2, null, -3); + final MockProcessorContext> context = new MockProcessorContext<>(props); + context.setRecordMetadata("left", -1, -2); join.init(context); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KTableKTableOuterJoin.class)) { - join.process(null, new Change<>("new", "old")); + join.process(new Record<>(null, new Change<>("new", "old"), -3)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java index 11d8cc80ab6f6..8de7e5d5d22d2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java @@ -20,8 +20,9 @@ import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.processor.MockProcessorContext; -import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.api.MockProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; @@ -47,23 +48,25 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKeyWithBuiltInMetricsVer shouldLogAndMeterSkippedRecordsDueToNullLeftKey(StreamsConfig.METRICS_LATEST); } + @SuppressWarnings("unchecked") private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtInMetricsVersion) { final StreamsBuilder builder = new StreamsBuilder(); - @SuppressWarnings("unchecked") - final Processor> join = new KTableKTableRightJoin<>( - (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())), - (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())), + final Processor, String, Change> join = new KTableKTableRightJoin<>( + (KTableImpl) builder + .table("left", Consumed.with(Serdes.String(), Serdes.String())), + (KTableImpl) builder + .table("right", Consumed.with(Serdes.String(), Serdes.String())), null ).get(); props.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); - final MockProcessorContext context = new MockProcessorContext(props); - context.setRecordMetadata("left", -1, -2, null, -3); + final MockProcessorContext> context = new MockProcessorContext<>(props); + context.setRecordMetadata("left", -1, -2); join.init(context); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KTableKTableRightJoin.class)) { - join.process(null, new Change<>("new", "old")); + join.process(new Record<>(null, new Change<>("new", "old"), -3)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java index 008effebb5ba8..db5818ca3278c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -34,6 +35,7 @@ public class KTableReduceTest { + @SuppressWarnings("unchecked") @Test public void shouldAddAndSubtract() { final InternalMockProcessorContext context = new InternalMockProcessorContext(); @@ -52,14 +54,11 @@ public void shouldAddAndSubtract() { reduceProcessor.init(context); context.setCurrentNode(new ProcessorNode<>("reduce", reduceProcessor, singleton("myStore"))); - context.setTime(10L); - reduceProcessor.process("A", new Change<>(singleton("a"), null)); + reduceProcessor.process(new Record<>("A", new Change<>(singleton("a"), null), 10L)); assertEquals(ValueAndTimestamp.make(singleton("a"), 10L), myStore.get("A")); - context.setTime(15L); - reduceProcessor.process("A", new Change<>(singleton("b"), singleton("a"))); + reduceProcessor.process(new Record<>("A", new Change<>(singleton("b"), singleton("a")), 15L)); assertEquals(ValueAndTimestamp.make(singleton("b"), 15L), myStore.get("A")); - context.setTime(12L); - reduceProcessor.process("A", new Change<>(null, singleton("b"))); + reduceProcessor.process(new Record<>("A", new Change<>(null, singleton("b")), 12L)); assertEquals(ValueAndTimestamp.make(emptySet(), 15L), myStore.get("A")); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java index 498e8edb68908..02d3ec9a92b6a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java @@ -17,15 +17,15 @@ package org.apache.kafka.streams.kstream.internals.foreignkeyjoin; -import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.internals.KTableValueGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; -import org.apache.kafka.streams.processor.MockProcessorContext; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; +import org.apache.kafka.streams.processor.api.MockProcessorContext; +import org.apache.kafka.streams.processor.api.MockProcessorContext.CapturedForward; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.Murmur3; import org.junit.Test; @@ -43,14 +43,15 @@ public class SubscriptionResolverJoinProcessorSupplierTest { private static final ValueJoiner JOINER = (value1, value2) -> "(" + value1 + "," + value2 + ")"; - private static class TestKTableValueGetterSupplier implements KTableValueGetterSupplier { + private static class TestKTableValueGetterSupplier implements KTableValueAndTimestampGetterSupplier { private final Map map = new HashMap<>(); @Override - public KTableValueGetter get() { - return new KTableValueGetter() { + public KTableValueAndTimestampGetter get() { + return new KTableValueAndTimestampGetter() { + @Override - public void init(final ProcessorContext context) { + public void init(ProcessorContext context) { } @Override @@ -83,15 +84,16 @@ public void shouldNotForwardWhenHashDoesNotMatch() { JOINER, leftJoin ); - final Processor> processor = processorSupplier.get(); - final MockProcessorContext context = new MockProcessorContext(); + final Processor, String, String> processor = + processorSupplier.get(); + final MockProcessorContext context = new MockProcessorContext<>(); processor.init(context); - context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0); + context.setRecordMetadata("topic", 0, 0); valueGetterSupplier.put("lhs1", "lhsValue"); final long[] oldHash = Murmur3.hash128(STRING_SERIALIZER.serialize("topic-join-resolver", "oldLhsValue")); - processor.process("lhs1", new SubscriptionResponseWrapper<>(oldHash, "rhsValue")); - final List forwarded = context.forwarded(); + processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(oldHash, "rhsValue"), 0)); + final List> forwarded = context.forwarded(); assertThat(forwarded, empty()); } @@ -108,15 +110,16 @@ public void shouldIgnoreUpdateWhenLeftHasBecomeNull() { JOINER, leftJoin ); - final Processor> processor = processorSupplier.get(); - final MockProcessorContext context = new MockProcessorContext(); + final Processor, String, String> processor = + processorSupplier.get(); + final MockProcessorContext context = new MockProcessorContext<>(); processor.init(context); - context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0); + context.setRecordMetadata("topic", 0, 0); valueGetterSupplier.put("lhs1", null); final long[] hash = Murmur3.hash128(STRING_SERIALIZER.serialize("topic-join-resolver", "lhsValue")); - processor.process("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue")); - final List forwarded = context.forwarded(); + processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue"), 0L)); + final List> forwarded = context.forwarded(); assertThat(forwarded, empty()); } @@ -133,17 +136,18 @@ public void shouldForwardWhenHashMatches() { JOINER, leftJoin ); - final Processor> processor = processorSupplier.get(); - final MockProcessorContext context = new MockProcessorContext(); + final Processor, String, String> processor = processorSupplier.get(); + final MockProcessorContext context = new MockProcessorContext<>(); processor.init(context); - context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0); + context.setRecordMetadata("topic", 0, 0); valueGetterSupplier.put("lhs1", "lhsValue"); final long[] hash = Murmur3.hash128(STRING_SERIALIZER.serialize("topic-join-resolver", "lhsValue")); - processor.process("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue")); - final List forwarded = context.forwarded(); + processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue"), 0)); + List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); - assertThat(forwarded.get(0).keyValue(), is(new KeyValue<>("lhs1", "(lhsValue,rhsValue)"))); + assertThat(forwarded.get(0).record().key(), is("lhs1")); + assertThat(forwarded.get(0).record().value(), is("(lhsValue,rhsValue)")); } @Test @@ -159,17 +163,19 @@ public void shouldEmitTombstoneForInnerJoinWhenRightIsNull() { JOINER, leftJoin ); - final Processor> processor = processorSupplier.get(); - final MockProcessorContext context = new MockProcessorContext(); + final Processor, String, String> processor = processorSupplier + .get(); + final MockProcessorContext context = new MockProcessorContext<>(); processor.init(context); - context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0); + context.setRecordMetadata("topic", 0, 0); valueGetterSupplier.put("lhs1", "lhsValue"); final long[] hash = Murmur3.hash128(STRING_SERIALIZER.serialize("topic-join-resolver", "lhsValue")); - processor.process("lhs1", new SubscriptionResponseWrapper<>(hash, null)); - final List forwarded = context.forwarded(); + processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, null), 0)); + final List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); - assertThat(forwarded.get(0).keyValue(), is(new KeyValue<>("lhs1", null))); + assertThat(forwarded.get(0).record().key(), is("lhs1")); + assertThat(forwarded.get(0).record().value(), is(null)); } @Test @@ -185,17 +191,19 @@ public void shouldEmitResultForLeftJoinWhenRightIsNull() { JOINER, leftJoin ); - final Processor> processor = processorSupplier.get(); - final MockProcessorContext context = new MockProcessorContext(); + final Processor, String, String> processor = processorSupplier + .get(); + final MockProcessorContext context = new MockProcessorContext<>(); processor.init(context); - context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0); + context.setRecordMetadata("topic", 0, 0); valueGetterSupplier.put("lhs1", "lhsValue"); final long[] hash = Murmur3.hash128(STRING_SERIALIZER.serialize("topic-join-resolver", "lhsValue")); - processor.process("lhs1", new SubscriptionResponseWrapper<>(hash, null)); - final List forwarded = context.forwarded(); + processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, null), 0)); + final List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); - assertThat(forwarded.get(0).keyValue(), is(new KeyValue<>("lhs1", "(lhsValue,null)"))); + assertThat(forwarded.get(0).record().key(), is("lhs1")); + assertThat(forwarded.get(0).record().value(), is("(lhsValue,null)")); } @Test @@ -211,16 +219,17 @@ public void shouldEmitTombstoneForLeftJoinWhenRightIsNullAndLeftIsNull() { JOINER, leftJoin ); - final Processor> processor = processorSupplier.get(); - final MockProcessorContext context = new MockProcessorContext(); + final Processor, String, String> processor = processorSupplier.get(); + final MockProcessorContext context = new MockProcessorContext<>(); processor.init(context); - context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0); + context.setRecordMetadata("topic", 0, 0); valueGetterSupplier.put("lhs1", null); final long[] hash = null; - processor.process("lhs1", new SubscriptionResponseWrapper<>(hash, null)); - final List forwarded = context.forwarded(); + processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, null), 0)); + final List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); - assertThat(forwarded.get(0).keyValue(), is(new KeyValue<>("lhs1", null))); + assertThat(forwarded.get(0).record().key(), is("lhs1")); + assertThat(forwarded.get(0).record().value(), is(null)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java index 3b46765fe20fe..e4da1d8123963 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java @@ -25,10 +25,11 @@ import org.apache.kafka.streams.kstream.Suppressed; import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.KTableImpl; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer; import org.apache.kafka.test.MockInternalProcessorContext; @@ -230,16 +231,16 @@ private void shouldRecordMetrics(final String builtInMetricsVersion) { .build(); final KTableImpl mock = EasyMock.mock(KTableImpl.class); - final Processor> processor = - new KTableSuppressProcessorSupplier<>( - (SuppressedInternal) Suppressed.untilTimeLimit(Duration.ofDays(100), maxRecords(1)), - storeName, - mock - ).get(); + final Processor, String, Change> processor = new KTableSuppressProcessorSupplier<>( + (SuppressedInternal) Suppressed + .untilTimeLimit(Duration.ofDays(100), maxRecords(1)), + storeName, + mock + ).get(); streamsConfig.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); - final MockInternalProcessorContext context = - new MockInternalProcessorContext(streamsConfig, TASK_ID, TestUtils.tempDirectory()); + final MockInternalProcessorContext> context = + new MockInternalProcessorContext<>(streamsConfig, TASK_ID, TestUtils.tempDirectory()); final Time time = new SystemTime(); context.setCurrentNode(new ProcessorNode("testNode")); context.setSystemTimeMs(time.milliseconds()); @@ -251,7 +252,7 @@ private void shouldRecordMetrics(final String builtInMetricsVersion) { context.setRecordMetadata("", 0, 0L, null, timestamp); final String key = "longKey"; final Change value = new Change<>(null, ARBITRARY_LONG); - processor.process(key, value); + processor.process(new Record<>(key, value, timestamp)); final MetricName evictionRateMetric = StreamsConfig.METRICS_0100_TO_24.equals(builtInMetricsVersion) ? evictionRateMetric0100To24 : evictionRateMetricLatest; @@ -282,7 +283,7 @@ private void shouldRecordMetrics(final String builtInMetricsVersion) { } context.setRecordMetadata("", 0, 1L, null, timestamp + 1); - processor.process("key", value); + processor.process(new Record<>("key", value, timestamp + 1)); { final Map metrics = context.metrics().metrics(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java index 0c50886bc288e..356721d035bd7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java @@ -16,6 +16,23 @@ */ package org.apache.kafka.streams.kstream.internals.suppress; +import static java.time.Duration.ZERO; +import static java.time.Duration.ofMillis; +import static org.apache.kafka.common.serialization.Serdes.Long; +import static org.apache.kafka.common.serialization.Serdes.String; +import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.maxBytes; +import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.maxRecords; +import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.unbounded; +import static org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit; +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; +import static org.apache.kafka.streams.kstream.WindowedSerdes.sessionWindowedSerdeFrom; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; + +import java.time.Duration; +import java.util.Collection; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; @@ -28,10 +45,12 @@ import org.apache.kafka.streams.kstream.internals.KTableImpl; import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.kstream.internals.TimeWindow; -import org.apache.kafka.streams.processor.api.MockProcessorContext; -import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.MockProcessorContext.CapturedForward; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.api.MockProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer; import org.apache.kafka.test.MockInternalProcessorContext; @@ -41,34 +60,18 @@ import org.hamcrest.Matcher; import org.junit.Test; -import java.time.Duration; -import java.util.Collection; - -import static java.time.Duration.ZERO; -import static java.time.Duration.ofMillis; -import static org.apache.kafka.common.serialization.Serdes.Long; -import static org.apache.kafka.common.serialization.Serdes.String; -import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.maxBytes; -import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.maxRecords; -import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.unbounded; -import static org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit; -import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; -import static org.apache.kafka.streams.kstream.WindowedSerdes.sessionWindowedSerdeFrom; -import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.fail; - public class KTableSuppressProcessorTest { private static final long ARBITRARY_LONG = 5L; private static final Change ARBITRARY_CHANGE = new Change<>(7L, 14L); private static class Harness { + private final Processor, ?, ?> processor; - private final MockInternalProcessorContext context; + private final MockInternalProcessorContext> context; + @SuppressWarnings("unchecked") Harness(final Suppressed suppressed, final Serde keySerde, final Serde valueSerde) { @@ -104,10 +107,10 @@ public void zeroTimeLimitShouldImmediatelyEmit() { context.setRecordMetadata("", 0, 0L, null, timestamp); final String key = "hey"; final Change value = ARBITRARY_CHANGE; - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, 5L)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -122,10 +125,10 @@ public void windowedZeroTimeLimitShouldImmediatelyEmit() { context.setRecordMetadata("", 0, 0L, null, timestamp); final Windowed key = new Windowed<>("hey", new TimeWindow(0L, 100L)); final Change value = ARBITRARY_CHANGE; - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -140,14 +143,14 @@ public void intermediateSuppressionShouldBufferAndEmitLater() { context.setRecordMetadata("topic", 0, 0, null, timestamp); final String key = "hey"; final Change value = new Change<>(null, 1L); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(0)); context.setRecordMetadata("topic", 0, 1, null, 1L); - harness.processor.process("tick", new Change<>(null, null)); + harness.processor.process(new Record<>("tick", new Change<>(null, null), 1L)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -164,7 +167,7 @@ public void finalResultsSuppressionShouldBufferAndEmitAtGraceExpiration() { context.setRecordMetadata("topic", 0, 0, null, recordTime); final Windowed key = new Windowed<>("hey", new TimeWindow(windowStart, windowEnd)); final Change value = ARBITRARY_CHANGE; - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, recordTime)); assertThat(context.forwarded(), hasSize(0)); // although the stream time is now 100, we have to wait 1 ms after the window *end* before we @@ -173,7 +176,9 @@ public void finalResultsSuppressionShouldBufferAndEmitAtGraceExpiration() { final long recordTime2 = 100L; final long windowEnd2 = 101L; context.setRecordMetadata("topic", 0, 1, null, recordTime2); - harness.processor.process(new Windowed<>("dummyKey1", new TimeWindow(windowStart2, windowEnd2)), ARBITRARY_CHANGE); + harness.processor.process( + new Record<>(new Windowed<>("dummyKey1", new TimeWindow(windowStart2, windowEnd2)), + ARBITRARY_CHANGE, recordTime2)); assertThat(context.forwarded(), hasSize(0)); // ok, now it's time to emit "hey" @@ -181,10 +186,12 @@ public void finalResultsSuppressionShouldBufferAndEmitAtGraceExpiration() { final long recordTime3 = 101L; final long windowEnd3 = 102L; context.setRecordMetadata("topic", 0, 1, null, recordTime3); - harness.processor.process(new Windowed<>("dummyKey2", new TimeWindow(windowStart3, windowEnd3)), ARBITRARY_CHANGE); + harness.processor.process( + new Record<>(new Windowed<>("dummyKey2", new TimeWindow(windowStart3, windowEnd3)), + ARBITRARY_CHANGE, recordTime3)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(recordTime)); } @@ -207,14 +214,16 @@ public void finalResultsWithZeroGraceShouldStillBufferUntilTheWindowEnd() { context.setRecordMetadata("", 0, 0L, null, timestamp); final Windowed key = new Windowed<>("hey", new TimeWindow(0, windowEnd)); final Change value = ARBITRARY_CHANGE; - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(0)); context.setRecordMetadata("", 0, 1L, null, windowEnd); - harness.processor.process(new Windowed<>("dummyKey", new TimeWindow(windowEnd, windowEnd + 100L)), ARBITRARY_CHANGE); + harness.processor.process( + new Record<>(new Windowed<>("dummyKey", new TimeWindow(windowEnd, windowEnd + 100L)), + ARBITRARY_CHANGE, windowEnd)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -229,10 +238,10 @@ public void finalResultsWithZeroGraceAtWindowEndShouldImmediatelyEmit() { context.setRecordMetadata("", 0, 0L, null, timestamp); final Windowed key = new Windowed<>("hey", new TimeWindow(0, 100L)); final Change value = ARBITRARY_CHANGE; - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -251,7 +260,7 @@ public void finalResultsShouldDropTombstonesForTimeWindows() { context.setRecordMetadata("", 0, 0L, null, timestamp); final Windowed key = new Windowed<>("hey", new TimeWindow(0, 100L)); final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(0)); } @@ -271,7 +280,7 @@ public void finalResultsShouldDropTombstonesForSessionWindows() { context.setRecordMetadata("", 0, 0L, null, timestamp); final Windowed key = new Windowed<>("hey", new SessionWindow(0L, 0L)); final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(0)); } @@ -290,10 +299,10 @@ public void suppressShouldNotDropTombstonesForTimeWindows() { context.setRecordMetadata("", 0, 0L, null, timestamp); final Windowed key = new Windowed<>("hey", new TimeWindow(0L, 100L)); final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -313,10 +322,10 @@ public void suppressShouldNotDropTombstonesForSessionWindows() { context.setRecordMetadata("", 0, 0L, null, timestamp); final Windowed key = new Windowed<>("hey", new SessionWindow(0L, 0L)); final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -336,10 +345,10 @@ public void suppressShouldNotDropTombstonesForKTable() { context.setRecordMetadata("", 0, 0L, null, timestamp); final String key = "hey"; final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -354,13 +363,13 @@ public void suppressShouldEmitWhenOverRecordCapacity() { context.setRecordMetadata("", 0, 0L, null, timestamp); final String key = "hey"; final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); context.setRecordMetadata("", 0, 1L, null, timestamp + 1); - harness.processor.process("dummyKey", value); + harness.processor.process(new Record<>("dummyKey", value, timestamp + 1)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -375,13 +384,13 @@ public void suppressShouldEmitWhenOverByteCapacity() { context.setRecordMetadata("", 0, 0L, null, timestamp); final String key = "hey"; final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); context.setRecordMetadata("", 0, 1L, null, timestamp + 1); - harness.processor.process("dummyKey", value); + harness.processor.process(new Record<>("dummyKey", value, timestamp + 1)); assertThat(context.forwarded(), hasSize(1)); - final MockProcessorContext.CapturedForward capturedForward = context.forwarded().get(0); + final CapturedForward capturedForward = context.forwarded().get(0); assertThat(capturedForward.keyValue(), is(new KeyValue<>(key, value))); assertThat(capturedForward.timestamp(), is(timestamp)); } @@ -397,11 +406,11 @@ public void suppressShouldShutDownWhenOverRecordCapacity() { context.setCurrentNode(new ProcessorNode("testNode")); final String key = "hey"; final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); context.setRecordMetadata("", 0, 1L, null, timestamp); try { - harness.processor.process("dummyKey", value); + harness.processor.process(new Record<>("dummyKey", value, timestamp)); fail("expected an exception"); } catch (final StreamsException e) { assertThat(e.getMessage(), containsString("buffer exceeded its max capacity")); @@ -419,11 +428,11 @@ public void suppressShouldShutDownWhenOverByteCapacity() { context.setCurrentNode(new ProcessorNode("testNode")); final String key = "hey"; final Change value = new Change<>(null, ARBITRARY_LONG); - harness.processor.process(key, value); + harness.processor.process(new Record<>(key, value, timestamp)); context.setRecordMetadata("", 0, 1L, null, timestamp); try { - harness.processor.process("dummyKey", value); + harness.processor.process(new Record<>("dummyKey", value, timestamp)); fail("expected an exception"); } catch (final StreamsException e) { assertThat(e.getMessage(), containsString("buffer exceeded its max capacity")); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java index 0476e707db979..692ae5c6ba2af 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java @@ -96,7 +96,7 @@ public String newStoreName(final String prefix) { null, GLOBAL_STORE_TOPIC_NAME, "processorName", - () -> ProcessorAdapter.adapt(new KTableSource<>(GLOBAL_STORE_NAME, GLOBAL_STORE_NAME).get())); + () -> new KTableSource<>(GLOBAL_STORE_NAME, GLOBAL_STORE_NAME).get()); baseDirectoryName = TestUtils.tempDirectory().getAbsolutePath(); final HashMap properties = new HashMap<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 84bdf51cc8893..4eba431442a25 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -25,8 +25,8 @@ import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.test.InternalMockProcessorContext; @@ -62,14 +62,14 @@ public void shouldThrowStreamsExceptionIfExceptionCaughtDuringClose() { assertThrows(StreamsException.class, () -> node.init(null)); } - private static class ExceptionalProcessor implements Processor { + private static class ExceptionalProcessor implements Processor { @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { throw new RuntimeException(); } @Override - public void process(final Object key, final Object value) { + public void process(Record record) { throw new RuntimeException(); } @@ -79,20 +79,18 @@ public void close() { } } - private static class NoOpProcessor implements Processor { + private static class NoOpProcessor implements Processor { @Override - public void init(final ProcessorContext context) { - + public void init(final ProcessorContext context) { } @Override - public void process(final Object key, final Object value) { - + public void process(Record record) { } + @Override public void close() { - } } @@ -110,8 +108,8 @@ private void testMetrics(final String builtInMetricsVersion) { final Metrics metrics = new Metrics(); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test-client", builtInMetricsVersion, new MockTime()); - final InternalMockProcessorContext context = new InternalMockProcessorContext(streamsMetrics); - final ProcessorNode node = new ProcessorNode<>("name", new NoOpProcessor(), Collections.emptySet()); + final InternalMockProcessorContext context = new InternalMockProcessorContext<>(streamsMetrics); + final ProcessorNode node = new ProcessorNode<>("name", new NoOpProcessor(), Collections.emptySet()); node.init(context); final String threadId = Thread.currentThread().getName(); @@ -182,11 +180,11 @@ public void testTopologyLevelClassCastException() { private static class ClassCastProcessor extends ExceptionalProcessor { @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { } @Override - public void process(final Object key, final Object value) { + public void process(final Record record) { throw new ClassCastException("Incompatible types simulation exception."); } } @@ -196,8 +194,8 @@ public void testTopologyLevelClassCastExceptionDirect() { final Metrics metrics = new Metrics(); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test-client", StreamsConfig.METRICS_LATEST, new MockTime()); - final InternalMockProcessorContext context = new InternalMockProcessorContext(streamsMetrics); - final ProcessorNode node = new ProcessorNode<>("name", new ClassCastProcessor(), Collections.emptySet()); + final InternalMockProcessorContext context = new InternalMockProcessorContext<>(streamsMetrics); + final ProcessorNode node = new ProcessorNode<>("name", new ClassCastProcessor(), Collections.emptySet()); node.init(context); final StreamsException se = assertThrows( StreamsException.class, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java index d82f6c9d1a855..e142f8ffb5215 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java @@ -44,7 +44,8 @@ import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.StreamJoined; -import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.test.StreamsTestUtils; import org.junit.After; @@ -257,7 +258,7 @@ private List> getKeyValues() { return keyValueList; } - private static class SimpleProcessor extends AbstractProcessor { + private static class SimpleProcessor implements Processor { final List valueList; @@ -266,8 +267,8 @@ private static class SimpleProcessor extends AbstractProcessor { } @Override - public void process(final String key, final String value) { - valueList.add(value); + public void process(Record record) { + valueList.add(record.value()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java index 8d937ef90dca9..ad9d011282ca7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java @@ -33,8 +33,8 @@ public class SinkNodeTest { private final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class); private final Serializer anySerializer = Serdes.ByteArray().serializer(); private final RecordCollector recordCollector = new MockRecordCollector(); - private final InternalMockProcessorContext context = new InternalMockProcessorContext<>(anyStateSerde, recordCollector); - private final SinkNode sink = new SinkNode<>("anyNodeName", + private final InternalMockProcessorContext context = new InternalMockProcessorContext<>(anyStateSerde, recordCollector); + private final SinkNode sink = new SinkNode<>("anyNodeName", new StaticTopicNameExtractor<>("any-output-topic"), anySerializer, anySerializer, null); // Used to verify that the correct exceptions are thrown if the compiler checks are bypassed diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java index 92e47194becbe..00c56472a3635 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java @@ -44,7 +44,7 @@ public class SourceNodeTest { @Test public void shouldProvideTopicHeadersAndDataToKeyDeserializer() { - final SourceNode sourceNode = new MockSourceNode<>(new TheDeserializer(), new TheDeserializer()); + final SourceNode sourceNode = new MockSourceNode<>(new TheDeserializer(), new TheDeserializer()); final RecordHeaders headers = new RecordHeaders(); final String deserializeKey = sourceNode.deserializeKey("topic", headers, "data".getBytes(StandardCharsets.UTF_8)); assertThat(deserializeKey, is("topic" + headers + "data")); @@ -52,7 +52,7 @@ public void shouldProvideTopicHeadersAndDataToKeyDeserializer() { @Test public void shouldProvideTopicHeadersAndDataToValueDeserializer() { - final SourceNode sourceNode = new MockSourceNode<>(new TheDeserializer(), new TheDeserializer()); + final SourceNode sourceNode = new MockSourceNode<>(new TheDeserializer(), new TheDeserializer()); final RecordHeaders headers = new RecordHeaders(); final String deserializedValue = sourceNode.deserializeValue("topic", headers, "data".getBytes(StandardCharsets.UTF_8)); assertThat(deserializedValue, is("topic" + headers + "data")); @@ -84,8 +84,8 @@ private void shouldExposeProcessMetrics(final String builtInMetricsVersion) { final Metrics metrics = new Metrics(); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test-client", builtInMetricsVersion, new MockTime()); - final InternalMockProcessorContext context = new InternalMockProcessorContext(streamsMetrics); - final SourceNode node = + final InternalMockProcessorContext context = new InternalMockProcessorContext<>(streamsMetrics); + final SourceNode node = new SourceNode<>(context.currentNode().name(), new TheDeserializer(), new TheDeserializer()); node.init(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 7192ab6a1f84f..ef9d766fd89e2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -620,10 +620,10 @@ public void shouldRecordE2ELatencyOnSourceNodeAndTerminalNodes() { // Create a processor that only forwards even keys to test the metrics at the source and terminal nodes final MockSourceNode evenKeyForwardingSourceNode = new MockSourceNode(intDeserializer, intDeserializer) { - InternalProcessorContext context; + InternalProcessorContext context; @Override - public void init(final InternalProcessorContext context) { + public void init(final InternalProcessorContext context) { this.context = context; super.init(context); } @@ -1505,6 +1505,7 @@ public void shouldNotCheckpointOffsetsOnCommitIfEosIsEnabled() { assertFalse(checkpointFile.exists()); } + @SuppressWarnings("unchecked") @Test public void shouldThrowIllegalStateExceptionIfCurrentNodeIsNotNullWhenPunctuateCalled() { task = createStatelessTask(createConfig("100"), StreamsConfig.METRICS_LATEST); @@ -1545,6 +1546,7 @@ public void shouldThrowIllegalStateExceptionOnScheduleIfCurrentNodeIsNull() { assertThrows(IllegalStateException.class, () -> task.schedule(1, PunctuationType.STREAM_TIME, timestamp -> { })); } + @SuppressWarnings("unchecked") @Test public void shouldNotThrowExceptionOnScheduleIfCurrentNodeIsNotNull() { task = createStatelessTask(createConfig("100"), StreamsConfig.METRICS_LATEST); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 82b1996046295..99e3c2909e3b1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -64,7 +64,11 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TaskMetadata; import org.apache.kafka.streams.processor.ThreadMetadata; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.assignment.ReferenceContainer; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; @@ -1808,16 +1812,17 @@ public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled() { public void shouldPunctuateActiveTask() { final List punctuatedStreamTime = new ArrayList<>(); final List punctuatedWallClockTime = new ArrayList<>(); - final org.apache.kafka.streams.processor.ProcessorSupplier punctuateProcessor = - () -> new org.apache.kafka.streams.processor.AbstractProcessor() { + final ProcessorSupplier punctuateProcessor = + () -> new ContextualProcessor() { @Override - public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { - context.schedule(Duration.ofMillis(100L), PunctuationType.STREAM_TIME, punctuatedStreamTime::add); - context.schedule(Duration.ofMillis(100L), PunctuationType.WALL_CLOCK_TIME, punctuatedWallClockTime::add); + public void process(Record record) { } @Override - public void process(final Object key, final Object value) {} + public void init(final ProcessorContext context) { + context.schedule(Duration.ofMillis(100L), PunctuationType.STREAM_TIME, punctuatedStreamTime::add); + context.schedule(Duration.ofMillis(100L), PunctuationType.WALL_CLOCK_TIME, punctuatedWallClockTime::add); + } }; internalStreamsBuilder.stream(Collections.singleton(topic1), consumed).process(punctuateProcessor); @@ -1893,13 +1898,8 @@ public void close() {} }; final List peekedContextTime = new ArrayList<>(); - final org.apache.kafka.streams.processor.ProcessorSupplier peekProcessor = - () -> new org.apache.kafka.streams.processor.AbstractProcessor() { - @Override - public void process(final Object key, final Object value) { - peekedContextTime.add(context.timestamp()); - } - }; + final ProcessorSupplier peekProcessor = + () -> (Processor) record -> peekedContextTime.add(record.timestamp()); internalStreamsBuilder.stream(Collections.singleton(topic1), consumed) .transform(punctuateProcessor) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index 38e98603c7be0..16d5ca6a9d88f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -25,7 +25,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; @@ -141,7 +141,7 @@ public class KeyValueStoreTestDriver { /** * Create a driver object that will have a {@link #context()} that records messages - * {@link ProcessorContext#forward(Object, Object) forwarded} by the store and that provides default serializers and + * {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) forwarded} by the store and that provides default serializers and * deserializers for the given built-in key and value types (e.g., {@code String.class}, {@code Integer.class}, * {@code Long.class}, and {@code byte[].class}). This can be used when store is created to rely upon the * ProcessorContext's default key and value serializers and deserializers. @@ -159,7 +159,7 @@ public static KeyValueStoreTestDriver create(final Class keyClas /** * Create a driver object that will have a {@link #context()} that records messages - * {@link ProcessorContext#forward(Object, Object) forwarded} by the store and that provides the specified serializers and + * {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) forwarded} by the store and that provides the specified serializers and * deserializers. This can be used when store is created to rely upon the ProcessorContext's default key and value serializers * and deserializers. * @@ -184,9 +184,10 @@ public static KeyValueStoreTestDriver create(final Serializer ke private final Set flushedRemovals = new HashSet<>(); private final List> restorableEntries = new LinkedList<>(); - private final InternalMockProcessorContext context; + private final InternalMockProcessorContext context; private final StateSerdes stateSerdes; + @SuppressWarnings("unchecked") private KeyValueStoreTestDriver(final StateSerdes serdes) { props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "application-id"); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java index a1344f675924e..33e527dd07e61 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java @@ -78,7 +78,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest { private final long windowSizeForTimeWindow = 500; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private AbstractRocksDBSegmentedBytesStore bytesStore; private File stateDir; private final Window[] windows = new Window[4]; @@ -124,7 +124,7 @@ public void before() { bytesStore = getBytesStore(); stateDir = TestUtils.tempDirectory(); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( stateDir, Serdes.String(), Serdes.Long(), @@ -400,7 +400,7 @@ private void shouldLogAndMeasureExpiredRecords(final String builtInMetricsVersio final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); streamsConfig.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); final AbstractRocksDBSegmentedBytesStore bytesStore = getBytesStore(); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), new StreamsConfig(streamsConfig) ); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java index 66b13c1ad434f..59812401e5881 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java @@ -60,7 +60,7 @@ public class CachingInMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest private final static String TOPIC = "topic"; private static final String CACHE_NAMESPACE = "0_0-store-name"; private final int maxCacheSizeBytes = 150; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private CachingKeyValueStore store; private KeyValueStore underlyingStore; private ThreadCache cache; @@ -74,7 +74,7 @@ public void setUp() { store = new CachingKeyValueStore(underlyingStore); store.setFlushListener(cacheFlushListener, false); cache = new ThreadCache(new LogContext("testCache "), maxCacheSizeBytes, new MockStreamsMetrics(new Metrics())); - context = new InternalMockProcessorContext(null, null, null, null, cache); + context = new InternalMockProcessorContext<>(null, null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(10, 0, 0, TOPIC, null)); store.init((StateStoreContext) context, null); } @@ -200,7 +200,7 @@ private void setUpCloseTests() { EasyMock.replay(underlyingStore); store = new CachingKeyValueStore(underlyingStore); cache = EasyMock.niceMock(ThreadCache.class); - context = new InternalMockProcessorContext(TestUtils.tempDirectory(), null, null, null, cache); + context = new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(10, 0, 0, TOPIC, null)); store.init((StateStoreContext) context, store); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java index e584e2ca706b6..66e8b4b751d88 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java @@ -78,7 +78,7 @@ public class CachingInMemorySessionStoreTest { private final Bytes keyB = Bytes.wrap("b".getBytes()); private SessionStore underlyingStore; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private CachingSessionStore cachingStore; private ThreadCache cache; @@ -87,7 +87,7 @@ public void before() { underlyingStore = new InMemorySessionStore("store-name", Long.MAX_VALUE, "metric-scope"); cachingStore = new CachingSessionStore(underlyingStore, SEGMENT_INTERVAL); cache = new ThreadCache(new LogContext("testCache "), MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); - context = new InternalMockProcessorContext(TestUtils.tempDirectory(), null, null, null, cache); + context = new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, TOPIC, null)); cachingStore.init((StateStoreContext) context, cachingStore); } @@ -223,7 +223,7 @@ private void setUpCloseTests() { EasyMock.replay(underlyingStore); cachingStore = new CachingSessionStore(underlyingStore, SEGMENT_INTERVAL); cache = EasyMock.niceMock(ThreadCache.class); - final InternalMockProcessorContext context = new InternalMockProcessorContext(TestUtils.tempDirectory(), null, null, null, cache); + final InternalMockProcessorContext context = new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(10, 0, 0, TOPIC, null)); cachingStore.init((StateStoreContext) context, cachingStore); } @@ -674,7 +674,8 @@ public void shouldNotThrowInvalidRangeExceptionWhenBackwardWithNegativeFromKey() final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingSessionStore.class)) { - final KeyValueIterator, byte[]> iterator = cachingStore.backwardFindSessions(keyFrom, keyTo, 0L, 10L); + final KeyValueIterator, byte[]> iterator = + cachingStore.backwardFindSessions(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); final List messages = appender.getMessages(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java index d472c7f5637db..a13abed1bf6e5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java @@ -91,8 +91,8 @@ public void before() { underlyingStore = new RocksDBSessionStore(segmented); cachingStore = new CachingSessionStore(underlyingStore, SEGMENT_INTERVAL); cache = new ThreadCache(new LogContext("testCache "), MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); - final InternalMockProcessorContext context = - new InternalMockProcessorContext(TestUtils.tempDirectory(), null, null, null, cache); + final InternalMockProcessorContext context = + new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, TOPIC, null)); cachingStore.init((StateStoreContext) context, cachingStore); } @@ -207,8 +207,8 @@ private void setUpCloseTests() { EasyMock.replay(underlyingStore); cachingStore = new CachingSessionStore(underlyingStore, SEGMENT_INTERVAL); cache = EasyMock.niceMock(ThreadCache.class); - final InternalMockProcessorContext context = - new InternalMockProcessorContext(TestUtils.tempDirectory(), null, null, null, cache); + final InternalMockProcessorContext context = + new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(10, 0, 0, TOPIC, null)); cachingStore.init((StateStoreContext) context, cachingStore); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java index 735101f8158d1..7564531fba92d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java @@ -84,7 +84,7 @@ public class CachingPersistentWindowStoreTest { private final static String TOPIC = "topic"; private static final String CACHE_NAMESPACE = "0_0-store-name"; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private RocksDBSegmentedBytesStore bytesStore; private WindowStore underlyingStore; private CachingWindowStore cachingStore; @@ -103,7 +103,7 @@ public void setUp() { cachingStore = new CachingWindowStore(underlyingStore, WINDOW_SIZE, SEGMENT_INTERVAL); cachingStore.setFlushListener(cacheListener, false); cache = new ThreadCache(new LogContext("testCache "), MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); - context = new InternalMockProcessorContext(TestUtils.tempDirectory(), null, null, null, cache); + context = new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, TOPIC, null)); cachingStore.init((StateStoreContext) context, cachingStore); } @@ -905,7 +905,7 @@ private void setUpCloseTests() { EasyMock.replay(underlyingStore); cachingStore = new CachingWindowStore(underlyingStore, WINDOW_SIZE, SEGMENT_INTERVAL); cache = EasyMock.createNiceMock(ThreadCache.class); - context = new InternalMockProcessorContext(TestUtils.tempDirectory(), null, null, null, cache); + context = new InternalMockProcessorContext<>(TestUtils.tempDirectory(), null, null, null, cache); context.setRecordContext(new ProcessorRecordContext(10, 0, 0, TOPIC, null)); cachingStore.init((StateStoreContext) context, cachingStore); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java index 255994ca33b44..f5121607c4918 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java @@ -58,13 +58,13 @@ public class ChangeLoggingKeyValueBytesStoreTest { @Before public void before() { - final InternalMockProcessorContext context = mockContext(); + final InternalMockProcessorContext context = mockContext(); context.setTime(0); store.init((StateStoreContext) context, store); } - private InternalMockProcessorContext mockContext() { - return new InternalMockProcessorContext( + private InternalMockProcessorContext mockContext() { + return new InternalMockProcessorContext<>( TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), @@ -81,7 +81,7 @@ public void after() { @SuppressWarnings("deprecation") @Test public void shouldDelegateDeprecatedInit() { - final InternalMockProcessorContext context = mockContext(); + final InternalMockProcessorContext context = mockContext(); final KeyValueStore innerMock = EasyMock.mock(InMemoryKeyValueStore.class); final StateStore outer = new ChangeLoggingKeyValueBytesStore(innerMock); innerMock.init((ProcessorContext) context, outer); @@ -93,7 +93,7 @@ public void shouldDelegateDeprecatedInit() { @Test public void shouldDelegateInit() { - final InternalMockProcessorContext context = mockContext(); + final InternalMockProcessorContext context = mockContext(); final KeyValueStore innerMock = EasyMock.mock(InMemoryKeyValueStore.class); final StateStore outer = new ChangeLoggingKeyValueBytesStore(innerMock); innerMock.init((StateStoreContext) context, outer); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStoreTest.java index 8295f7d2e4af8..88c52031d0366 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStoreTest.java @@ -58,13 +58,13 @@ public class ChangeLoggingTimestampedKeyValueBytesStoreTest { @Before public void before() { - final InternalMockProcessorContext context = mockContext(); + final InternalMockProcessorContext context = mockContext(); context.setTime(0); store.init((StateStoreContext) context, store); } - private InternalMockProcessorContext mockContext() { - return new InternalMockProcessorContext( + private InternalMockProcessorContext mockContext() { + return new InternalMockProcessorContext<>( TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), @@ -81,7 +81,7 @@ public void after() { @SuppressWarnings("deprecation") @Test public void shouldDelegateDeprecatedInit() { - final InternalMockProcessorContext context = mockContext(); + final InternalMockProcessorContext context = mockContext(); final KeyValueStore inner = EasyMock.mock(InMemoryKeyValueStore.class); final StateStore outer = new ChangeLoggingTimestampedKeyValueBytesStore(inner); inner.init((ProcessorContext) context, outer); @@ -93,7 +93,7 @@ public void shouldDelegateDeprecatedInit() { @Test public void shouldDelegateInit() { - final InternalMockProcessorContext context = mockContext(); + final InternalMockProcessorContext context = mockContext(); final KeyValueStore inner = EasyMock.mock(InMemoryKeyValueStore.class); final StateStore outer = new ChangeLoggingTimestampedKeyValueBytesStore(inner); inner.init((StateStoreContext) context, outer); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java index 736721ac48bdb..bdd1073c5cc5e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java @@ -76,7 +76,8 @@ private KeyValueStore newStoreInstance() { Serdes.String()) .build(); - final InternalMockProcessorContext context = new InternalMockProcessorContext(new StateSerdes<>(ProcessorStateManager.storeChangelogTopic("appId", storeName), + final InternalMockProcessorContext context = + new InternalMockProcessorContext<>(new StateSerdes<>(ProcessorStateManager.storeChangelogTopic("appId", storeName), Serdes.String(), Serdes.String()), new MockRecordCollector()); context.setTime(1L); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java index aeef8ce8e13f5..43a8a4ec6de85 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java @@ -47,7 +47,7 @@ public class KeyValueSegmentsTest { private static final long SEGMENT_INTERVAL = 100L; private static final long RETENTION_PERIOD = 4 * SEGMENT_INTERVAL; private static final String METRICS_SCOPE = "test-state-id"; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private KeyValueSegments segments; private File stateDirectory; private final String storeName = "test"; @@ -55,7 +55,7 @@ public class KeyValueSegmentsTest { @Before public void createContext() { stateDirectory = TestUtils.tempDirectory(); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( stateDirectory, Serdes.String(), Serdes.Long(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java index c05c1ba5e67c3..c90f1998aa139 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java @@ -66,7 +66,7 @@ public class MeteredTimestampedWindowStoreTest { private static final byte[] VALUE_AND_TIMESTAMP_BYTES = "\0\0\0\0\0\0\0avalue".getBytes(); private static final int WINDOW_SIZE_MS = 10; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private final WindowStore innerStoreMock = EasyMock.createNiceMock(WindowStore.class); private final Metrics metrics = new Metrics(new MetricConfig().recordLevel(Sensor.RecordingLevel.DEBUG)); private MeteredTimestampedWindowStore store = new MeteredTimestampedWindowStore<>( @@ -87,7 +87,7 @@ public void setUp() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", StreamsConfig.METRICS_LATEST, new MockTime()); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index cb3a268cace15..ae8c5e1c23662 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -33,7 +33,6 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; @@ -98,7 +97,7 @@ public class MeteredWindowStoreTest { private static final long TIMESTAMP = 42L; private final String threadId = Thread.currentThread().getName(); - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private final WindowStore innerStoreMock = createNiceMock(WindowStore.class); private MeteredWindowStore store = new MeteredWindowStore<>( innerStoreMock, @@ -132,7 +131,7 @@ public static Collection data() { public void setUp() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", builtInMetricsVersion, new MockTime()); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), @@ -318,7 +317,8 @@ public void shouldRecordPutLatency() { @Test public void shouldRecordFetchLatency() { - expect(innerStoreMock.fetch(Bytes.wrap("a".getBytes()), 1, 1)).andReturn(KeyValueIterators.emptyWindowStoreIterator()); + expect(innerStoreMock.fetch(Bytes.wrap("a".getBytes()), 1, 1)) + .andReturn(KeyValueIterators.emptyWindowStoreIterator()); replay(innerStoreMock); store.init((StateStoreContext) context, store); @@ -343,7 +343,8 @@ public void shouldRecordFetchLatency() { @Test public void shouldRecordFetchRangeLatency() { - expect(innerStoreMock.fetch(Bytes.wrap("a".getBytes()), Bytes.wrap("b".getBytes()), 1, 1)).andReturn(KeyValueIterators., byte[]>emptyIterator()); + expect(innerStoreMock.fetch(Bytes.wrap("a".getBytes()), Bytes.wrap("b".getBytes()), 1, 1)) + .andReturn(KeyValueIterators.emptyIterator()); replay(innerStoreMock); store.init((StateStoreContext) context, store); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index faf659bf231d6..4991a9fdeac0f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -128,14 +128,14 @@ private RocksDBStore getRocksDBStoreWithRocksDBMetricsRecorder() { return new RocksDBStore(DB_NAME, METRICS_SCOPE, metricsRecorder); } - private InternalMockProcessorContext getProcessorContext(final Properties streamsProps) { - return new InternalMockProcessorContext( + private InternalMockProcessorContext getProcessorContext(final Properties streamsProps) { + return new InternalMockProcessorContext<>( TestUtils.tempDirectory(), new StreamsConfig(streamsProps) ); } - private InternalMockProcessorContext getProcessorContext( + private InternalMockProcessorContext getProcessorContext( final RecordingLevel recordingLevel, final Class rocksDBConfigSetterClass) { @@ -145,7 +145,7 @@ private InternalMockProcessorContext getProcessorContext( return getProcessorContext(streamsProps); } - private InternalMockProcessorContext getProcessorContext(final RecordingLevel recordingLevel) { + private InternalMockProcessorContext getProcessorContext(final RecordingLevel recordingLevel) { final Properties streamsProps = StreamsTestUtils.getStreamsConfig(); streamsProps.setProperty(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, recordingLevel.name()); return getProcessorContext(streamsProps); @@ -320,7 +320,7 @@ public void shouldCallRocksDbConfigSetter() { @Test public void shouldThrowProcessorStateExceptionOnOpeningReadOnlyDir() { final File tmpDir = TestUtils.tempDirectory(); - final InternalMockProcessorContext tmpContext = new InternalMockProcessorContext(tmpDir, new StreamsConfig(StreamsTestUtils.getStreamsConfig())); + final InternalMockProcessorContext tmpContext = new InternalMockProcessorContext<>(tmpDir, new StreamsConfig(StreamsTestUtils.getStreamsConfig())); assertTrue(tmpDir.setReadOnly()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java index 558f1c9471b1e..b06a0983ce58f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java @@ -47,7 +47,7 @@ public class TimestampedSegmentsTest { private static final long SEGMENT_INTERVAL = 100L; private static final long RETENTION_PERIOD = 4 * SEGMENT_INTERVAL; private static final String METRICS_SCOPE = "test-state-id"; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private TimestampedSegments segments; private File stateDirectory; private final String storeName = "test"; @@ -55,7 +55,7 @@ public class TimestampedSegmentsTest { @Before public void createContext() { stateDirectory = TestUtils.tempDirectory(); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( stateDirectory, Serdes.String(), Serdes.Long(), diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index e8ec04c23ca52..afc5992810ed8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -23,70 +23,65 @@ import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import java.time.Instant; +import org.apache.kafka.streams.processor.api.Record; public class SmokeTestUtil { final static int END = Integer.MAX_VALUE; - static ProcessorSupplier printProcessorSupplier(final String topic) { + static ProcessorSupplier printProcessorSupplier(final String topic) { return printProcessorSupplier(topic, ""); } - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return new ProcessorSupplier() { + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return () -> new ContextualProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(Record record) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + if (smallestOffset > context().recordMetadata().get().offset()) { + smallestOffset = context().recordMetadata().get().offset(); + } + if (largestOffset < context().recordMetadata().get().offset()) { + largestOffset = context().recordMetadata().get().offset(); + } + } + @Override - public Processor get() { - return new AbstractProcessor() { - private int numRecordsProcessed = 0; - private long smallestOffset = Long.MAX_VALUE; - private long largestOffset = Long.MIN_VALUE; - - @Override - public void init(final ProcessorContext context) { - super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); - System.out.flush(); - numRecordsProcessed = 0; - smallestOffset = Long.MAX_VALUE; - largestOffset = Long.MIN_VALUE; - } - - @Override - public void process(final Object key, final Object value) { - numRecordsProcessed++; - if (numRecordsProcessed % 100 == 0) { - System.out.printf("%s: %s%n", name, Instant.now()); - System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); - } - - if (smallestOffset > context().offset()) { - smallestOffset = context().offset(); - } - if (largestOffset < context().offset()) { - largestOffset = context().offset(); - } - } - - @Override - public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); - System.out.println("processed " + numRecordsProcessed + " records"); - final long processed; - if (largestOffset >= smallestOffset) { - processed = 1L + largestOffset - smallestOffset; - } else { - processed = 0L; - } - System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); - System.out.flush(); - } - }; + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); } }; } diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java index c32c136d4f451..b9618bef24d88 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java @@ -40,7 +40,7 @@ import java.util.Properties; import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener; -public class MockInternalProcessorContext extends MockProcessorContext implements InternalProcessorContext { +public class MockInternalProcessorContext extends MockProcessorContext implements InternalProcessorContext { private final Map restoreCallbacks = new LinkedHashMap<>(); private ProcessorNode currentNode; @@ -71,12 +71,13 @@ public StreamsMetricsImpl metrics() { } @Override - public void forward(final Record record) { + public void forward(Record record) { forward(record.key(), record.value(), To.all().withTimestamp(record.timestamp())); } @Override - public void forward(final Record record, final String childName) { + public void forward(Record record, + String childName) { forward(record.key(), record.value(), To.child(childName).withTimestamp(record.timestamp())); } From d1f29adc4a0ffcecca9418a4e01177e46bc53730 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 18:39:16 +0100 Subject: [PATCH 20/40] adjust kstream process --- .../DeserializationExceptionHandler.java | 2 +- .../apache/kafka/streams/kstream/KStream.java | 10 ++-- .../KStreamAggProcessorSupplier.java | 27 ---------- .../kstream/internals/KStreamImpl.java | 54 ++++++++++--------- .../streams/kstream/internals/KTableImpl.java | 2 +- ...KTableMaterializedValueGetterSupplier.java | 53 ------------------ .../internals/KTableProcessorSupplier.java | 40 -------------- .../KTableSourceValueGetterSupplier.java | 52 ------------------ .../kstream/internals/KTableValueGetter.java | 29 ---------- .../internals/KTableValueGetterSupplier.java | 24 --------- ...criptionResolverJoinProcessorSupplier.java | 6 +-- .../graph/StatefulProcessorNode.java | 8 ++- .../kstream/internals/KStreamImplTest.java | 9 ++-- ...amSessionWindowAggregateProcessorTest.java | 1 + 14 files changed, 45 insertions(+), 272 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java index d931177a98429..ca84f5bb71801 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java @@ -32,7 +32,7 @@ public interface DeserializationExceptionHandler extends Configurable { * Inspect a record and the exception received. *

* Note, that the passed in {@link ProcessorContext} only allows to access metadata like the task ID. - * However, it cannot be used to emit records via {@link ProcessorContext#forward(Record) ()}; + * However, it cannot be used to emit records via {@link ProcessorContext#forward(Record)}; * calling {@code forward()} (and some other methods) would result in a runtime exception. * * @param context processor context diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 7a89ddc33732f..e58114a83b37c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -4615,8 +4615,8 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier processorSupplier, -// final String... stateStoreNames); + void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, + final String... stateStoreNames); /** * Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given @@ -4709,9 +4709,9 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier processorSupplier, -// final Named named, -// final String... stateStoreNames); + void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, + final Named named, + final String... stateStoreNames); void process(final ProcessorSupplier processorSupplier, final String... stateStoreNames); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java deleted file mode 100644 index 3f2186052371b..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.ProcessorSupplier; - -public interface KStreamAggProcessorSupplier extends ProcessorSupplier { - - KTableValueGetterSupplier view(); - - void enableSendingOldValues(); -} - diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 923992a0849ba..0cf6f44cacad0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -71,6 +71,7 @@ import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopicProperties; +import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor; import org.apache.kafka.streams.state.KeyValueStore; @@ -1473,32 +1474,33 @@ private KStream doFlatTransformValues(final ValueTransformerWithKeyS builder); } -// @Override -// public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, -// final String... stateStoreNames) { -// process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); -// } -// -// @Override -// public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, -// final Named named, -// final String... stateStoreNames) { -// Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); -// Objects.requireNonNull(named, "named can't be null"); -// Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); -// ApiUtils.checkSupplier(processorSupplier); -// for (final String stateStoreName : stateStoreNames) { -// Objects.requireNonNull(stateStoreName, "stateStoreNames can't be null"); -// } -// -// final String name = new NamedInternal(named).name(); -// final StatefulProcessorNode processNode = new StatefulProcessorNode<>( -// name, -// new ProcessorParameters<>(processorSupplier, name), -// stateStoreNames); -// -// builder.addGraphNode(graphNode, processNode); -// } + @Override + public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, + final String... stateStoreNames) { + process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); + } + + @Override + public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, + final Named named, + final String... stateStoreNames) { + Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); + Objects.requireNonNull(named, "named can't be null"); + Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); + ApiUtils.checkSupplier(processorSupplier); + for (final String stateStoreName : stateStoreNames) { + Objects.requireNonNull(stateStoreName, "stateStoreNames can't be null"); + } + + final String name = new NamedInternal(named).name(); + final ProcessorSupplier processor = () -> ProcessorAdapter.adaptRaw(processorSupplier.get()); + final StatefulProcessorNode processNode = new StatefulProcessorNode<>( + name, + new ProcessorParameters<>(processor, name), + stateStoreNames); + + builder.addGraphNode(graphNode, processNode); + } @Override public void process(final ProcessorSupplier processorSupplier, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index b97a87e3b5ed7..e35d039ece8ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -830,7 +830,7 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { return false; } source.enableSendingOldValues(); - } else if (processorSupplier instanceof KStreamAggProcessorSupplier) { + } else if (processorSupplier instanceof KStreamAggregateProcessorSupplier) { ((KStreamAggregateProcessorSupplier) processorSupplier).enableSendingOldValues(); } else { final KTableChangeProcessorSupplier tableProcessorSupplier = diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java deleted file mode 100644 index 4d121a3383cf4..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.state.TimestampedKeyValueStore; -import org.apache.kafka.streams.state.ValueAndTimestamp; - -public class KTableMaterializedValueGetterSupplier implements KTableValueGetterSupplier { - private final String storeName; - - KTableMaterializedValueGetterSupplier(final String storeName) { - this.storeName = storeName; - } - - public KTableValueGetter get() { - return new KTableMaterializedValueGetter(); - } - - @Override - public String[] storeNames() { - return new String[]{storeName}; - } - - private class KTableMaterializedValueGetter implements KTableValueGetter { - private TimestampedKeyValueStore store; - - @SuppressWarnings("unchecked") - @Override - public void init(final ProcessorContext context) { - store = (TimestampedKeyValueStore) context.getStateStore(storeName); - } - - @Override - public ValueAndTimestamp get(final K key) { - return store.get(key); - } - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java deleted file mode 100644 index ff6f9d61df969..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.ProcessorSupplier; - -public interface KTableProcessorSupplier extends ProcessorSupplier> { - - KTableValueGetterSupplier view(); - - /** - * Potentially enables sending old values. - *

- * If {@code forceMaterialization} is {@code true}, the method will force the materialization of upstream nodes to - * enable sending old values. - *

- * If {@code forceMaterialization} is {@code false}, the method will only enable the sending of old values if - * an upstream node is already materialized. - * - * @param forceMaterialization indicates if an upstream node should be forced to materialize to enable sending old - * values. - * @return {@code true} is sending old values is enabled, i.e. either because {@code forceMaterialization} was - * {@code true} or some upstream node is materialized. - */ - boolean enableSendingOldValues(boolean forceMaterialization); -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java deleted file mode 100644 index f7b087b6641a8..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.state.TimestampedKeyValueStore; -import org.apache.kafka.streams.state.ValueAndTimestamp; - -public class KTableSourceValueGetterSupplier implements KTableValueGetterSupplier { - private final String storeName; - - public KTableSourceValueGetterSupplier(final String storeName) { - this.storeName = storeName; - } - - public KTableValueGetter get() { - return new KTableSourceValueGetter(); - } - - @Override - public String[] storeNames() { - return new String[]{storeName}; - } - - private class KTableSourceValueGetter implements KTableValueGetter { - private TimestampedKeyValueStore store = null; - - @SuppressWarnings("unchecked") - public void init(final ProcessorContext context) { - store = (TimestampedKeyValueStore) context.getStateStore(storeName); - } - - public ValueAndTimestamp get(final K key) { - return store.get(key); - } - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java deleted file mode 100644 index 12145fa335fcb..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.state.ValueAndTimestamp; - -public interface KTableValueGetter { - - void init(ProcessorContext context); - - ValueAndTimestamp get(K key); - - default void close() {} -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java deleted file mode 100644 index aa28e9ab5213e..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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; - -public interface KTableValueGetterSupplier { - - KTableValueGetter get(); - - String[] storeNames(); -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java index 084bacef55de5..a4a17eb594cf7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java @@ -17,14 +17,12 @@ package org.apache.kafka.streams.kstream.internals.foreignkeyjoin; +import java.util.function.Supplier; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; -import org.apache.kafka.streams.kstream.internals.KTableValueGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; @@ -33,8 +31,6 @@ import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.Murmur3; -import java.util.function.Supplier; - /** * Receives {@code SubscriptionResponseWrapper} events and filters out events which do not match the current hash * of the primary key. This eliminates race-condition results for rapidly-changing foreign-keys for a given primary key. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java index 3d6a9b15bb196..46f9834ae3dba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java @@ -16,16 +16,14 @@ */ package org.apache.kafka.streams.kstream.internals.graph; +import java.util.Arrays; +import java.util.Set; +import java.util.stream.Stream; import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; -import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.StoreBuilder; -import java.util.Arrays; -import java.util.Set; -import java.util.stream.Stream; - public class StatefulProcessorNode extends ProcessorGraphNode { private final String[] storeNames; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 6787e2390e4d9..605d935233b05 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -55,6 +55,7 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.SourceNode; @@ -2381,7 +2382,7 @@ public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueWithKeySupp public void shouldNotAllowNullProcessSupplierOnProcess() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null)); + () -> testStream.process((ProcessorSupplier) null)); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2389,7 +2390,7 @@ public void shouldNotAllowNullProcessSupplierOnProcess() { public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null, "storeName")); + () -> testStream.process((ProcessorSupplier) null, "storeName")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2397,7 +2398,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null, Named.as("processor"))); + () -> testStream.process((ProcessorSupplier) null, Named.as("processor"))); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2405,7 +2406,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamedAndStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process(null, Named.as("processor"), "stateStore")); + () -> testStream.process((ProcessorSupplier) null, Named.as("processor"), "stateStore")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index 632796f0d01d2..c2775a0fc9555 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -109,6 +109,7 @@ public void initializeStore() { new ThreadCache(new LogContext("testCache "), 100000, metrics), Time.SYSTEM ) { + @SuppressWarnings("unchecked") @Override public void forward(final Object key, final Object value, final To to) { toInternal.update(to); From c5fb7eb7a1d1b2136590ba74e2cb181d07da0064 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 18:46:52 +0100 Subject: [PATCH 21/40] clean --- .../AggregationProcessorSupplier.java | 36 ------------------- .../streams/kstream/internals/KTableImpl.java | 2 +- 2 files changed, 1 insertion(+), 37 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java deleted file mode 100644 index 027384569cc5c..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AggregationProcessorSupplier.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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; - -import org.apache.kafka.streams.processor.api.ProcessorSupplier; - -/** - * - * @param - * @param - * @param - * @param Table tuple - * @param - */ -public interface AggregationProcessorSupplier extends - ProcessorSupplier> { - - KTableValueAndTimestampGetterSupplier view(); - - void enableSendingOldValues(); -} - diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index e35d039ece8ef..7ebdf68d35382 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -815,7 +815,7 @@ public KTableValueAndTimestampGetterSupplier valueAndTimestampGetterSuppli source.materialize(); return new KTableSourceValueAndTimestampGetterSupplier<>(source.queryableName()); } else if (processorSupplier instanceof KStreamAggregateProcessorSupplier) { - return ((AggregationProcessorSupplier) processorSupplier).view(); + return ((KStreamAggregateProcessorSupplier) processorSupplier).view(); } else { return ((KTableChangeProcessorSupplier) processorSupplier).view(); } From fa524471eeaf6d61825ea178bbb25b2031320486 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 18:54:09 +0100 Subject: [PATCH 22/40] renames --- .../kstream/internals/GlobalKTableImpl.java | 6 ++--- .../internals/InternalStreamsBuilder.java | 2 +- .../kstream/internals/KStreamAggregate.java | 8 +++---- .../KStreamAggregateProcessorSupplier.java | 3 +-- .../internals/KStreamGlobalKTableJoin.java | 4 ++-- .../kstream/internals/KStreamImpl.java | 6 ++--- .../kstream/internals/KStreamKTableJoin.java | 4 ++-- .../internals/KStreamKTableJoinProcessor.java | 4 ++-- .../kstream/internals/KStreamReduce.java | 8 +++---- .../KStreamSessionWindowAggregate.java | 8 +++---- .../KStreamSlidingWindowAggregate.java | 8 +++---- .../internals/KStreamWindowAggregate.java | 8 +++---- .../kstream/internals/KTableAggregate.java | 4 ++-- .../KTableChangeProcessorSupplier.java | 2 +- .../kstream/internals/KTableFilter.java | 18 +++++++-------- .../streams/kstream/internals/KTableImpl.java | 14 +++++------ .../internals/KTableKTableAbstractJoin.java | 8 +++---- ...KTableAbstractJoinValueGetterSupplier.java | 11 +++++---- .../internals/KTableKTableInnerJoin.java | 22 +++++++++--------- .../internals/KTableKTableJoinMerger.java | 8 +++---- .../internals/KTableKTableLeftJoin.java | 22 +++++++++--------- .../internals/KTableKTableOuterJoin.java | 22 +++++++++--------- .../internals/KTableKTableRightJoin.java | 22 +++++++++--------- .../kstream/internals/KTableMapValues.java | 18 +++++++-------- ...TableMaterializedValueGetterSupplier.java} | 11 +++++---- .../kstream/internals/KTablePassThrough.java | 8 +++---- .../kstream/internals/KTableReduce.java | 4 ++-- .../internals/KTableRepartitionMap.java | 16 ++++++------- ...a => KTableSourceValueGetterSupplier.java} | 8 +++---- .../internals/KTableTransformValues.java | 16 ++++++------- ...tampGetter.java => KTableValueGetter.java} | 2 +- ...er.java => KTableValueGetterSupplier.java} | 4 ++-- ...scriptionJoinForeignProcessorSupplier.java | 10 ++++---- ...criptionResolverJoinProcessorSupplier.java | 10 ++++---- .../graph/StatefulProcessorNode.java | 4 ++-- .../KTableSuppressProcessorSupplier.java | 16 ++++++------- ...amSessionWindowAggregateProcessorTest.java | 2 +- .../kstream/internals/KTableFilterTest.java | 8 +++---- .../internals/KTableMapValuesTest.java | 8 +++---- .../kstream/internals/KTableSourceTest.java | 4 ++-- .../internals/KTableTransformValuesTest.java | 23 +++++++++---------- ...tionResolverJoinProcessorSupplierTest.java | 11 +++++---- 42 files changed, 203 insertions(+), 202 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/kstream/internals/{KTableMaterializedValueAndTimestampGetterSupplier.java => KTableMaterializedValueGetterSupplier.java} (78%) rename streams/src/main/java/org/apache/kafka/streams/kstream/internals/{KTableSourceValueAndTimestampGetterSupplier.java => KTableSourceValueGetterSupplier.java} (85%) rename streams/src/main/java/org/apache/kafka/streams/kstream/internals/{KTableValueAndTimestampGetter.java => KTableValueGetter.java} (95%) rename streams/src/main/java/org/apache/kafka/streams/kstream/internals/{KTableValueAndTimestampGetterSupplier.java => KTableValueGetterSupplier.java} (88%) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java index 10f2658c27d35..734ff4a17e711 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java @@ -20,16 +20,16 @@ public class GlobalKTableImpl implements GlobalKTable { - private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; + private final KTableValueGetterSupplier valueGetterSupplier; private final String queryableStoreName; - GlobalKTableImpl(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, + GlobalKTableImpl(final KTableValueGetterSupplier valueGetterSupplier, final String queryableStoreName) { this.valueGetterSupplier = valueGetterSupplier; this.queryableStoreName = queryableStoreName; } - KTableValueAndTimestampGetterSupplier valueGetterSupplier() { + KTableValueGetterSupplier valueGetterSupplier() { return valueGetterSupplier; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java index 514dfbec80849..6fa65c13894d4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java @@ -185,7 +185,7 @@ public GlobalKTable globalTable(final String topic, addGraphNode(root, tableSourceNode); - return new GlobalKTableImpl<>(new KTableSourceValueAndTimestampGetterSupplier<>(storeName), materialized.queryableStoreName()); + return new GlobalKTableImpl<>(new KTableSourceValueGetterSupplier<>(storeName), materialized.queryableStoreName()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index ed7338eb680cd..b13c46ef7da96 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -111,10 +111,10 @@ public void process(Record record) { } @Override - public KTableValueAndTimestampGetterSupplier view() { - return new KTableValueAndTimestampGetterSupplier() { + public KTableValueGetterSupplier view() { + return new KTableValueGetterSupplier() { - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KStreamAggregateValueGetter(); } @@ -126,7 +126,7 @@ public String[] storeNames() { } - private class KStreamAggregateValueGetter implements KTableValueAndTimestampGetter { + private class KStreamAggregateValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java index ee0364a867f60..bd64c8a2293b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java @@ -20,8 +20,7 @@ public interface KStreamAggregateProcessorSupplier extends ProcessorSupplier> { - KTableValueAndTimestampGetterSupplier view(); + KTableValueGetterSupplier view(); void enableSendingOldValues(); } - diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java index 9156ddd29a655..0c56a49a1d744 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java @@ -23,12 +23,12 @@ class KStreamGlobalKTableJoin implements ProcessorSupplier { - private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; + private final KTableValueGetterSupplier valueGetterSupplier; private final ValueJoinerWithKey joiner; private final KeyValueMapper mapper; private final boolean leftJoin; - KStreamGlobalKTableJoin(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, + KStreamGlobalKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, final ValueJoinerWithKey joiner, final KeyValueMapper mapper, final boolean leftJoin) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 0cf6f44cacad0..e13ae57460c41 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1210,7 +1210,7 @@ private KStream globalTableJoin(final GlobalKTable g Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(named, "named can't be null"); - final KTableValueAndTimestampGetterSupplier valueGetterSupplier = + final KTableValueGetterSupplier valueGetterSupplier = ((GlobalKTableImpl) globalTable).valueGetterSupplier(); final String name = new NamedInternal(named) .orElseGenerateWithPrefix(builder, LEFTJOIN_NAME); @@ -1252,7 +1252,7 @@ private KStream doStreamTableJoin(final KTable table, final String name = renamed.orElseGenerateWithPrefix(builder, leftJoin ? LEFTJOIN_NAME : JOIN_NAME); final KStreamKTableJoin processorSupplier = new KStreamKTableJoin<>( - ((KTableImpl) table).valueAndTimestampGetterSupplier(), + ((KTableImpl) table).valueGetterSupplier(), joiner, leftJoin); @@ -1261,7 +1261,7 @@ private KStream doStreamTableJoin(final KTable table, final StreamTableJoinNode streamTableJoinNode = new StreamTableJoinNode<>( name, processorParameters, - ((KTableImpl) table).valueAndTimestampGetterSupplier().storeNames(), + ((KTableImpl) table).valueGetterSupplier().storeNames(), this.name ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java index bc85a884a6abd..631fad581f7fe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java @@ -24,11 +24,11 @@ class KStreamKTableJoin implements ProcessorSupplier { private final KeyValueMapper keyValueMapper = (key, value) -> key; - private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; + private final KTableValueGetterSupplier valueGetterSupplier; private final ValueJoinerWithKey joiner; private final boolean leftJoin; - KStreamKTableJoin(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, + KStreamKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, final ValueJoinerWithKey joiner, final boolean leftJoin) { this.valueGetterSupplier = valueGetterSupplier; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 7aab6139ebbfc..62d6e75829488 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -32,14 +32,14 @@ class KStreamKTableJoinProcessor extends ContextualProcessor { private static final Logger LOG = LoggerFactory.getLogger(KStreamKTableJoin.class); - private final KTableValueAndTimestampGetter valueGetter; + private final KTableValueGetter valueGetter; private final KeyValueMapper keyMapper; private final ValueJoinerWithKey joiner; private final boolean leftJoin; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KStreamKTableJoinProcessor(final KTableValueAndTimestampGetter valueGetter, + KStreamKTableJoinProcessor(final KTableValueGetter valueGetter, final KeyValueMapper keyMapper, final ValueJoinerWithKey joiner, final boolean leftJoin) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index df90e3cdeae9b..afb97c83c7e8c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -104,10 +104,10 @@ public void process(Record record) { } @Override - public KTableValueAndTimestampGetterSupplier view() { - return new KTableValueAndTimestampGetterSupplier() { + public KTableValueGetterSupplier view() { + return new KTableValueGetterSupplier() { - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KStreamReduceValueGetter(); } @@ -119,7 +119,7 @@ public String[] storeNames() { } - private class KStreamReduceValueGetter implements KTableValueAndTimestampGetter { + private class KStreamReduceValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index 7eea3e95c6b5d..e0a702781c3ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -186,10 +186,10 @@ private SessionWindow mergeSessionWindow(final SessionWindow one, final SessionW } @Override - public KTableValueAndTimestampGetterSupplier, Agg> view() { - return new KTableValueAndTimestampGetterSupplier, Agg>() { + public KTableValueGetterSupplier, Agg> view() { + return new KTableValueGetterSupplier, Agg>() { @Override - public KTableValueAndTimestampGetter, Agg> get() { + public KTableValueGetter, Agg> get() { return new KTableSessionWindowValueGetter(); } @@ -201,7 +201,7 @@ public String[] storeNames() { } private class KTableSessionWindowValueGetter - implements KTableValueAndTimestampGetter, Agg> { + implements KTableValueGetter, Agg> { private SessionStore store; @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index a6fc9f7fc987a..52bb37dfee4d9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -504,10 +504,10 @@ private void updateWindowAndForward(final Window window, } @Override - public KTableValueAndTimestampGetterSupplier, Agg> view() { - return new KTableValueAndTimestampGetterSupplier, Agg>() { + public KTableValueGetterSupplier, Agg> view() { + return new KTableValueGetterSupplier, Agg>() { - public KTableValueAndTimestampGetter, Agg> get() { + public KTableValueGetter, Agg> get() { return new KStreamWindowAggregateValueGetter(); } @@ -519,7 +519,7 @@ public String[] storeNames() { } private class KStreamWindowAggregateValueGetter - implements KTableValueAndTimestampGetter, Agg> { + implements KTableValueGetter, Agg> { private TimestampedWindowStore windowStore; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 40368e850d9af..0d05aa493f03e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -175,10 +175,10 @@ public void process(Record record) { } @Override - public KTableValueAndTimestampGetterSupplier, Agg> view() { - return new KTableValueAndTimestampGetterSupplier, Agg>() { + public KTableValueGetterSupplier, Agg> view() { + return new KTableValueGetterSupplier, Agg>() { - public KTableValueAndTimestampGetter, Agg> get() { + public KTableValueGetter, Agg> get() { return new KStreamWindowAggregateValueGetter(); } @@ -190,7 +190,7 @@ public String[] storeNames() { } - private class KStreamWindowAggregateValueGetter implements KTableValueAndTimestampGetter, Agg> { + private class KStreamWindowAggregateValueGetter implements KTableValueGetter, Agg> { private TimestampedWindowStore windowStore; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index ce409a3acc203..31c9e6b6fb616 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -120,7 +120,7 @@ public void process(Record> record) { } @Override - public KTableValueAndTimestampGetterSupplier view() { - return new KTableMaterializedValueAndTimestampGetterSupplier<>(storeName); + public KTableValueGetterSupplier view() { + return new KTableMaterializedValueGetterSupplier<>(storeName); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java index 674a692292025..2e91e5b0e5dae 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java @@ -20,7 +20,7 @@ public interface KTableChangeProcessorSupplier extends ProcessorSupplier, KOut, Change> { - KTableValueAndTimestampGetterSupplier view(); + KTableValueGetterSupplier view(); /** * Potentially enables sending old values. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java index f84c4ace453cf..925bb06ba2600 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java @@ -133,17 +133,17 @@ private V computeOldValue(final K key, final Change change) { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { // if the KTable is materialized, use the materialized store to return getter value; // otherwise rely on the parent getter and apply filter on-the-fly if (queryableName != null) { - return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); + return new KTableMaterializedValueGetterSupplier<>(queryableName); } else { - return new KTableValueAndTimestampGetterSupplier() { - final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = - parent.valueAndTimestampGetterSupplier(); + return new KTableValueGetterSupplier() { + final KTableValueGetterSupplier parentValueGetterSupplier = + parent.valueGetterSupplier(); - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableFilterValueGetter(parentValueGetterSupplier.get()); } @@ -156,10 +156,10 @@ public String[] storeNames() { } - private class KTableFilterValueGetter implements KTableValueAndTimestampGetter { - private final KTableValueAndTimestampGetter parentGetter; + private class KTableFilterValueGetter implements KTableValueGetter { + private final KTableValueGetter parentGetter; - KTableFilterValueGetter(final KTableValueAndTimestampGetter parentGetter) { + KTableFilterValueGetter(final KTableValueGetter parentGetter) { this.parentGetter = parentGetter; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 7ebdf68d35382..5bd02858da365 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -753,9 +753,9 @@ private KTable doJoin(final KTable other, .withJoinOtherProcessorParameters(joinOtherProcessorParameters) .withThisJoinSideNodeName(name) .withOtherJoinSideNodeName(((KTableImpl) other).name) - .withJoinThisStoreNames(valueAndTimestampGetterSupplier().storeNames()) + .withJoinThisStoreNames(valueGetterSupplier().storeNames()) .withJoinOtherStoreNames( - ((KTableImpl) other).valueAndTimestampGetterSupplier().storeNames()) + ((KTableImpl) other).valueGetterSupplier().storeNames()) .withKeySerde(keySerde) .withValueSerde(valueSerde) .withQueryableStoreName(queryableStoreName) @@ -808,12 +808,12 @@ public KGroupedTable groupBy(final KeyValueMapper valueAndTimestampGetterSupplier() { + public KTableValueGetterSupplier valueGetterSupplier() { if (processorSupplier instanceof KTableSource) { final KTableSource source = (KTableSource) processorSupplier; // whenever a source ktable is required for getter, it should be materialized source.materialize(); - return new KTableSourceValueAndTimestampGetterSupplier<>(source.queryableName()); + return new KTableSourceValueGetterSupplier<>(source.queryableName()); } else if (processorSupplier instanceof KStreamAggregateProcessorSupplier) { return ((KStreamAggregateProcessorSupplier) processorSupplier).view(); } else { @@ -1068,13 +1068,13 @@ private KTable doJoinOnForeignKey(final KTable forei new StatefulProcessorNode<>( new ProcessorParameters<>( new SubscriptionJoinForeignProcessorSupplier<>( - ((KTableImpl) foreignKeyTable).valueAndTimestampGetterSupplier() + ((KTableImpl) foreignKeyTable).valueGetterSupplier() ), renamed.suffixWithOrElseGet("-subscription-join-foreign", builder, SUBSCRIPTION_PROCESSOR) ), Collections.emptySet(), Collections.singleton( - ((KTableImpl) foreignKeyTable).valueAndTimestampGetterSupplier()) + ((KTableImpl) foreignKeyTable).valueGetterSupplier()) ); builder.addGraphNode(subscriptionReceiveNode, subscriptionJoinForeignNode); @@ -1113,7 +1113,7 @@ private KTable doJoinOnForeignKey(final KTable forei resultSourceNodes.add(foreignResponseSource.nodeName()); builder.internalTopologyBuilder.copartitionSources(resultSourceNodes); - final KTableValueAndTimestampGetterSupplier primaryKeyValueGetter = valueAndTimestampGetterSupplier(); + final KTableValueGetterSupplier primaryKeyValueGetter = valueGetterSupplier(); final SubscriptionResolverJoinProcessorSupplier resolverProcessorSupplier = new SubscriptionResolverJoinProcessorSupplier<>( primaryKeyValueGetter, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java index 4d693218fb4b9..89ff90a585d33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java @@ -22,8 +22,8 @@ abstract class KTableKTableAbstractJoin implements KTableChangePro private final KTableImpl table1; private final KTableImpl table2; - final KTableValueAndTimestampGetterSupplier valueGetterSupplier1; - final KTableValueAndTimestampGetterSupplier valueGetterSupplier2; + final KTableValueGetterSupplier valueGetterSupplier1; + final KTableValueGetterSupplier valueGetterSupplier2; final ValueJoiner joiner; boolean sendOldValues = false; @@ -33,8 +33,8 @@ abstract class KTableKTableAbstractJoin implements KTableChangePro final ValueJoiner joiner) { this.table1 = table1; this.table2 = table2; - this.valueGetterSupplier1 = table1.valueAndTimestampGetterSupplier(); - this.valueGetterSupplier2 = table2.valueAndTimestampGetterSupplier(); + this.valueGetterSupplier1 = table1.valueGetterSupplier(); + this.valueGetterSupplier2 = table2.valueGetterSupplier(); this.joiner = joiner; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java index cb0b5fa47b5e1..20b3c3a9fde69 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java @@ -20,12 +20,13 @@ import java.util.HashSet; import java.util.Set; -public abstract class KTableKTableAbstractJoinValueGetterSupplier implements KTableValueAndTimestampGetterSupplier { - final KTableValueAndTimestampGetterSupplier valueGetterSupplier1; - final KTableValueAndTimestampGetterSupplier valueGetterSupplier2; +public abstract class KTableKTableAbstractJoinValueGetterSupplier implements + KTableValueGetterSupplier { + final KTableValueGetterSupplier valueGetterSupplier1; + final KTableValueGetterSupplier valueGetterSupplier2; - KTableKTableAbstractJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, - final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { + KTableKTableAbstractJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { this.valueGetterSupplier1 = valueGetterSupplier1; this.valueGetterSupplier2 = valueGetterSupplier2; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index d4c9f109dc813..74d4c728ec74b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -48,7 +48,7 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableInnerJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } @@ -57,12 +57,12 @@ private class KTableKTableInnerJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { KTableKTableInnerJoinValueGetterSupplier( - final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, - final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { + final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableKTableInnerJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } @@ -70,11 +70,11 @@ public KTableValueAndTimestampGetter get() { private class KTableKTableJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueAndTimestampGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { + KTableKTableJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @@ -128,13 +128,13 @@ public void close() { } } - private class KTableKTableInnerJoinValueGetter implements KTableValueAndTimestampGetter { + private class KTableKTableInnerJoinValueGetter implements KTableValueGetter { - private final KTableValueAndTimestampGetter valueGetter1; - private final KTableValueAndTimestampGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableInnerJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, - final KTableValueAndTimestampGetter valueGetter2) { + KTableKTableInnerJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java index e6244aa4ec785..4db2167d057b8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java @@ -52,15 +52,15 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { // if the result KTable is materialized, use the materialized store to return getter value; // otherwise rely on the parent getter and apply join on-the-fly if (queryableName != null) { - return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); + return new KTableMaterializedValueGetterSupplier<>(queryableName); } else { - return new KTableValueAndTimestampGetterSupplier() { + return new KTableValueGetterSupplier() { - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return parent1.view().get(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 295f7cc089870..9b35a7cee2c24 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -46,18 +46,18 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableLeftJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } private class KTableKTableLeftJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableLeftJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, - final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { + KTableKTableLeftJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableKTableLeftJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } @@ -65,11 +65,11 @@ public KTableValueAndTimestampGetter get() { private class KTableKTableLeftJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueAndTimestampGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableLeftJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { + KTableKTableLeftJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @@ -130,13 +130,13 @@ public void close() { } } - private class KTableKTableLeftJoinValueGetter implements KTableValueAndTimestampGetter { + private class KTableKTableLeftJoinValueGetter implements KTableValueGetter { - private final KTableValueAndTimestampGetter valueGetter1; - private final KTableValueAndTimestampGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableLeftJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, - final KTableValueAndTimestampGetter valueGetter2) { + KTableKTableLeftJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index c35a0b1babffb..f0a41e7378a1d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -46,29 +46,29 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableOuterJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } private class KTableKTableOuterJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableOuterJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, - final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { + KTableKTableOuterJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableKTableOuterJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } private class KTableKTableOuterJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueAndTimestampGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableOuterJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { + KTableKTableOuterJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @@ -125,13 +125,13 @@ public void close() { } } - private class KTableKTableOuterJoinValueGetter implements KTableValueAndTimestampGetter { + private class KTableKTableOuterJoinValueGetter implements KTableValueGetter { - private final KTableValueAndTimestampGetter valueGetter1; - private final KTableValueAndTimestampGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableOuterJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, - final KTableValueAndTimestampGetter valueGetter2) { + KTableKTableOuterJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index eba22c87fc0f2..c583d5a1460c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -45,29 +45,29 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableRightJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } private class KTableKTableRightJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableRightJoinValueGetterSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier1, - final KTableValueAndTimestampGetterSupplier valueGetterSupplier2) { + KTableKTableRightJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableKTableRightJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } private class KTableKTableRightJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueAndTimestampGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableRightJoinProcessor(final KTableValueAndTimestampGetter valueGetter) { + KTableKTableRightJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @@ -120,13 +120,13 @@ public void close() { } } - private class KTableKTableRightJoinValueGetter implements KTableValueAndTimestampGetter { + private class KTableKTableRightJoinValueGetter implements KTableValueGetter { - private final KTableValueAndTimestampGetter valueGetter1; - private final KTableValueAndTimestampGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableRightJoinValueGetter(final KTableValueAndTimestampGetter valueGetter1, - final KTableValueAndTimestampGetter valueGetter2) { + KTableKTableRightJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java index 5fb6605093cf0..350ab8cc27df4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java @@ -46,17 +46,17 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { // if the KTable is materialized, use the materialized store to return getter value; // otherwise rely on the parent getter and apply map-values on-the-fly if (queryableName != null) { - return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); + return new KTableMaterializedValueGetterSupplier<>(queryableName); } else { - return new KTableValueAndTimestampGetterSupplier() { - final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = - parent.valueAndTimestampGetterSupplier(); + return new KTableValueGetterSupplier() { + final KTableValueGetterSupplier parentValueGetterSupplier = + parent.valueGetterSupplier(); - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableMapValuesValueGetter(parentValueGetterSupplier.get()); } @@ -147,10 +147,10 @@ private VOut computeOldValue(final K key, final Change change) { } - private class KTableMapValuesValueGetter implements KTableValueAndTimestampGetter { - private final KTableValueAndTimestampGetter parentGetter; + private class KTableMapValuesValueGetter implements KTableValueGetter { + private final KTableValueGetter parentGetter; - KTableMapValuesValueGetter(final KTableValueAndTimestampGetter parentGetter) { + KTableMapValuesValueGetter(final KTableValueGetter parentGetter) { this.parentGetter = parentGetter; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueAndTimestampGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java similarity index 78% rename from streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueAndTimestampGetterSupplier.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java index a111759f7ac2d..fa085717e5a04 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueAndTimestampGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java @@ -20,15 +20,16 @@ import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; -public class KTableMaterializedValueAndTimestampGetterSupplier implements KTableValueAndTimestampGetterSupplier { +public class KTableMaterializedValueGetterSupplier implements + KTableValueGetterSupplier { private final String storeName; - KTableMaterializedValueAndTimestampGetterSupplier(final String storeName) { + KTableMaterializedValueGetterSupplier(final String storeName) { this.storeName = storeName; } - public KTableValueAndTimestampGetter get() { - return new KTableMaterializedValueAndTimestampGetter(); + public KTableValueGetter get() { + return new KTableMaterializedValueGetter(); } @Override @@ -36,7 +37,7 @@ public String[] storeNames() { return new String[]{storeName}; } - private class KTableMaterializedValueAndTimestampGetter implements KTableValueAndTimestampGetter { + private class KTableMaterializedValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java index e2140df80ca33..6224337795bc3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java @@ -50,11 +50,11 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { - return new KTableValueAndTimestampGetterSupplier() { + return new KTableValueGetterSupplier() { - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTablePassThroughValueGetter(); } @@ -72,7 +72,7 @@ public void process(final Record> record) { } } - private class KTablePassThroughValueGetter implements KTableValueAndTimestampGetter { + private class KTablePassThroughValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index 6bc386a1873ff..b3126881f0ecb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -111,7 +111,7 @@ public void process(Record> record) { } @Override - public KTableValueAndTimestampGetterSupplier view() { - return new KTableMaterializedValueAndTimestampGetterSupplier<>(storeName); + public KTableValueGetterSupplier view() { + return new KTableMaterializedValueGetterSupplier<>(storeName); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index 750ae056eadbb..5ecfffec9976b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -52,13 +52,13 @@ public Processor, K1, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier> view() { - final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = - parent.valueAndTimestampGetterSupplier(); + public KTableValueGetterSupplier> view() { + final KTableValueGetterSupplier parentValueGetterSupplier = + parent.valueGetterSupplier(); - return new KTableValueAndTimestampGetterSupplier>() { + return new KTableValueGetterSupplier>() { - public KTableValueAndTimestampGetter> get() { + public KTableValueGetter> get() { return new KTableMapValueGetter(parentValueGetterSupplier.get()); } @@ -114,10 +114,10 @@ public void process(Record> record) { } } - private class KTableMapValueGetter implements KTableValueAndTimestampGetter> { - private final KTableValueAndTimestampGetter parentGetter; + private class KTableMapValueGetter implements KTableValueGetter> { + private final KTableValueGetter parentGetter; - KTableMapValueGetter(final KTableValueAndTimestampGetter parentGetter) { + KTableMapValueGetter(final KTableValueGetter parentGetter) { this.parentGetter = parentGetter; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueAndTimestampGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java similarity index 85% rename from streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueAndTimestampGetterSupplier.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java index 4eb10a8207869..337add01e6f1d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueAndTimestampGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java @@ -21,14 +21,14 @@ import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; -public class KTableSourceValueAndTimestampGetterSupplier implements KTableValueAndTimestampGetterSupplier { +public class KTableSourceValueGetterSupplier implements KTableValueGetterSupplier { private final String storeName; - public KTableSourceValueAndTimestampGetterSupplier(final String storeName) { + public KTableSourceValueGetterSupplier(final String storeName) { this.storeName = storeName; } - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableSourceValueGetter(); } @@ -37,7 +37,7 @@ public String[] storeNames() { return new String[]{storeName}; } - private class KTableSourceValueGetter implements KTableValueAndTimestampGetter { + private class KTableSourceValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store = null; @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java index 9bbdf6b0ad975..27fa802854a6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java @@ -50,15 +50,15 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { + public KTableValueGetterSupplier view() { if (queryableName != null) { - return new KTableMaterializedValueAndTimestampGetterSupplier<>(queryableName); + return new KTableMaterializedValueGetterSupplier<>(queryableName); } - return new KTableValueAndTimestampGetterSupplier() { - final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = parent.valueAndTimestampGetterSupplier(); + return new KTableValueGetterSupplier() { + final KTableValueGetterSupplier parentValueGetterSupplier = parent.valueGetterSupplier(); - public KTableValueAndTimestampGetter get() { + public KTableValueGetter get() { return new KTableTransformValuesGetter( parentValueGetterSupplier.get(), transformerSupplier.get()); @@ -130,11 +130,11 @@ public void close() { } - private class KTableTransformValuesGetter implements KTableValueAndTimestampGetter { - private final KTableValueAndTimestampGetter parentGetter; + private class KTableTransformValuesGetter implements KTableValueGetter { + private final KTableValueGetter parentGetter; private final ValueTransformerWithKey valueTransformer; - KTableTransformValuesGetter(final KTableValueAndTimestampGetter parentGetter, + KTableTransformValuesGetter(final KTableValueGetter parentGetter, final ValueTransformerWithKey valueTransformer) { this.parentGetter = Objects.requireNonNull(parentGetter, "parentGetter"); this.valueTransformer = Objects.requireNonNull(valueTransformer, "valueTransformer"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java similarity index 95% rename from streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetter.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java index b3ba7de5f8cd3..38fcc325ccb10 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java @@ -19,7 +19,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.state.ValueAndTimestamp; -public interface KTableValueAndTimestampGetter { +public interface KTableValueGetter { void init(ProcessorContext context); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java similarity index 88% rename from streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetterSupplier.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java index 89a8b998b19d1..aa28e9ab5213e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueAndTimestampGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java @@ -16,9 +16,9 @@ */ package org.apache.kafka.streams.kstream.internals; -public interface KTableValueAndTimestampGetterSupplier { +public interface KTableValueGetterSupplier { - KTableValueAndTimestampGetter get(); + KTableValueGetter get(); String[] storeNames(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java index be5fe70924e18..9da1ca373f91b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java @@ -20,8 +20,8 @@ import java.util.Objects; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; +import org.apache.kafka.streams.kstream.internals.KTableValueGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; @@ -42,10 +42,10 @@ public class SubscriptionJoinForeignProcessorSupplier implements ProcessorSupplier, Change>>, CombinedKey, SubscriptionResponseWrapper> { - private final KTableValueAndTimestampGetterSupplier foreignValueGetterSupplier; + private final KTableValueGetterSupplier foreignValueGetterSupplier; public SubscriptionJoinForeignProcessorSupplier( - final KTableValueAndTimestampGetterSupplier foreignValueGetterSupplier) { + final KTableValueGetterSupplier foreignValueGetterSupplier) { this.foreignValueGetterSupplier = foreignValueGetterSupplier; } @@ -54,7 +54,7 @@ public Processor, Change, Change>>, CombinedKey, SubscriptionResponseWrapper>() { - private KTableValueAndTimestampGetter foreignValues; + private KTableValueGetter foreignValues; @Override public void init( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java index a4a17eb594cf7..e0e9c10fbec97 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java @@ -21,8 +21,8 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; +import org.apache.kafka.streams.kstream.internals.KTableValueGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; @@ -42,13 +42,13 @@ * @param Type of joined result of primary and foreign values */ public class SubscriptionResolverJoinProcessorSupplier implements ProcessorSupplier, K, VR> { - private final KTableValueAndTimestampGetterSupplier valueGetterSupplier; + private final KTableValueGetterSupplier valueGetterSupplier; private final Serializer constructionTimeValueSerializer; private final Supplier valueHashSerdePseudoTopicSupplier; private final ValueJoiner joiner; private final boolean leftJoin; - public SubscriptionResolverJoinProcessorSupplier(final KTableValueAndTimestampGetterSupplier valueGetterSupplier, + public SubscriptionResolverJoinProcessorSupplier(final KTableValueGetterSupplier valueGetterSupplier, final Serializer valueSerializer, final Supplier valueHashSerdePseudoTopicSupplier, final ValueJoiner joiner, @@ -66,7 +66,7 @@ public Processor, K, VR> get() { private String valueHashSerdePseudoTopic; private Serializer runtimeValueSerializer = constructionTimeValueSerializer; - private KTableValueAndTimestampGetter valueGetter; + private KTableValueGetter valueGetter; @SuppressWarnings("unchecked") @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java index 46f9834ae3dba..4df2fc9ba0ffd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java @@ -19,7 +19,7 @@ import java.util.Arrays; import java.util.Set; import java.util.stream.Stream; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; +import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.StoreBuilder; @@ -34,7 +34,7 @@ public class StatefulProcessorNode extends ProcessorGraphNode { */ public StatefulProcessorNode(final ProcessorParameters processorParameters, final Set> preRegisteredStores, - final Set> valueGetterSuppliers) { + final Set> valueGetterSuppliers) { super(processorParameters.processorName(), processorParameters); final Stream registeredStoreNames = preRegisteredStores.stream().map(StoreBuilder::name); final Stream valueGetterStoreNames = valueGetterSuppliers.stream().flatMap(s -> Arrays.stream(s.storeNames())); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java index 44fc5f01ca269..ebbb79b695b6c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java @@ -25,8 +25,8 @@ import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.KTableChangeProcessorSupplier; import org.apache.kafka.streams.kstream.internals.KTableImpl; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; +import org.apache.kafka.streams.kstream.internals.KTableValueGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.kstream.internals.suppress.TimeDefinitions.TimeDefinition; import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; @@ -62,14 +62,14 @@ public Processor, K, Change> get() { } @Override - public KTableValueAndTimestampGetterSupplier view() { - final KTableValueAndTimestampGetterSupplier parentValueGetterSupplier = parentKTable.valueAndTimestampGetterSupplier(); - return new KTableValueAndTimestampGetterSupplier() { + public KTableValueGetterSupplier view() { + final KTableValueGetterSupplier parentValueGetterSupplier = parentKTable.valueGetterSupplier(); + return new KTableValueGetterSupplier() { @Override - public KTableValueAndTimestampGetter get() { - final KTableValueAndTimestampGetter parentGetter = parentValueGetterSupplier.get(); - return new KTableValueAndTimestampGetter() { + public KTableValueGetter get() { + final KTableValueGetter parentGetter = parentValueGetterSupplier.get(); + return new KTableValueGetter() { private TimeOrderedKeyValueBuffer buffer; @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index c2775a0fc9555..c3883dd129bc6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -289,7 +289,7 @@ public void shouldHandleMultipleSessionsAndMerging() { @Test public void shouldGetAggregatedValuesFromValueGetter() { - final KTableValueAndTimestampGetter, Long> getter = sessionAggregator + final KTableValueGetter, Long> getter = sessionAggregator .view().get(); getter.init(context); processor.process(new Record<>("a", "1", 0)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index ec0ffd2eaad2b..321c6a00897de 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -140,8 +140,8 @@ private void doTestValueGetter(final StreamsBuilder builder, final Topology topology = builder.build(); - final KTableValueAndTimestampGetterSupplier getterSupplier2 = table2.valueAndTimestampGetterSupplier(); - final KTableValueAndTimestampGetterSupplier getterSupplier3 = table3.valueAndTimestampGetterSupplier(); + final KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); + final KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology); topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames()); @@ -151,8 +151,8 @@ private void doTestValueGetter(final StreamsBuilder builder, final TestInputTopic inputTopic = driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final KTableValueAndTimestampGetter getter2 = getterSupplier2.get(); - final KTableValueAndTimestampGetter getter3 = getterSupplier3.get(); + final KTableValueGetter getter2 = getterSupplier2.get(); + final KTableValueGetter getter3 = getterSupplier3.get(); getter2.init(driver.setCurrentNodeForProcessorContext(table2.name)); getter3.init(driver.setCurrentNodeForProcessorContext(table3.name)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index c1c6993f9558a..8fee802e58405 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -110,8 +110,8 @@ private void doTestValueGetter(final StreamsBuilder builder, final Topology topology = builder.build(); - final KTableValueAndTimestampGetterSupplier getterSupplier2 = table2.valueAndTimestampGetterSupplier(); - final KTableValueAndTimestampGetterSupplier getterSupplier3 = table3.valueAndTimestampGetterSupplier(); + final KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); + final KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology); topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames()); @@ -120,8 +120,8 @@ private void doTestValueGetter(final StreamsBuilder builder, try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(builder.build(), props)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final KTableValueAndTimestampGetter getter2 = getterSupplier2.get(); - final KTableValueAndTimestampGetter getter3 = getterSupplier3.get(); + final KTableValueGetter getter2 = getterSupplier2.get(); + final KTableValueGetter getter3 = getterSupplier3.get(); getter2.init(driver.setCurrentNodeForProcessorContext(table2.name)); getter3.init(driver.setCurrentNodeForProcessorContext(table3.name)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 4d5611576bbd2..a409e19d13af0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -218,7 +218,7 @@ public void testValueGetter() { (KTableImpl) builder.table(topic1, stringConsumed, Materialized.as("store")); final Topology topology = builder.build(); - final KTableValueAndTimestampGetterSupplier getterSupplier1 = table1.valueAndTimestampGetterSupplier(); + final KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology); topologyBuilder.connectProcessorAndStateStores(table1.name, getterSupplier1.storeNames()); @@ -232,7 +232,7 @@ public void testValueGetter() { Instant.ofEpochMilli(0L), Duration.ZERO ); - final KTableValueAndTimestampGetter getter1 = getterSupplier1.get(); + final KTableValueGetter getter1 = getterSupplier1.get(); getter1.init(driver.setCurrentNodeForProcessorContext(table1.name)); inputTopic1.pipeInput("A", "01", 10L); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java index 67b7df1c5de89..e39ca7bdc2f92 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java @@ -63,7 +63,6 @@ import static org.easymock.EasyMock.verify; import static org.hamcrest.CoreMatchers.hasItems; import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.isA; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; @@ -85,9 +84,9 @@ public class KTableTransformValuesTest { @Mock(MockType.NICE) private InternalProcessorContext context; @Mock(MockType.NICE) - private KTableValueAndTimestampGetterSupplier parentGetterSupplier; + private KTableValueGetterSupplier parentGetterSupplier; @Mock(MockType.NICE) - private KTableValueAndTimestampGetter parentGetter; + private KTableValueGetter parentGetter; @Mock(MockType.NICE) private TimestampedKeyValueStore stateStore; @Mock(MockType.NICE) @@ -124,7 +123,7 @@ public void shouldThrowOnGetIfSupplierReturnsNull() { @Test public void shouldThrowOnViewGetIfSupplierReturnsNull() { - final KTableValueAndTimestampGetterSupplier view = + final KTableValueGetterSupplier view = new KTableTransformValues<>(parent, new NullSupplier(), null).view(); try { @@ -217,12 +216,12 @@ public void shouldTransformOnGetIfNotMaterialized() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); - expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); expect(parentGetterSupplier.get()).andReturn(parentGetter); expect(parentGetter.get("Key")).andReturn(ValueAndTimestamp.make("Value", -1L)); replay(parent, parentGetterSupplier, parentGetter); - final KTableValueAndTimestampGetter getter = transformValues.view().get(); + final KTableValueGetter getter = transformValues.view().get(); getter.init(context); final String result = getter.get("Key").value(); @@ -240,7 +239,7 @@ public void shouldGetFromStateStoreIfMaterialized() { expect(stateStore.get("Key")).andReturn(ValueAndTimestamp.make("something", 0L)); replay(context, stateStore); - final KTableValueAndTimestampGetter getter = transformValues.view().get(); + final KTableValueGetter getter = transformValues.view().get(); getter.init(context); final String result = getter.get("Key").value(); @@ -253,7 +252,7 @@ public void shouldGetStoreNamesFromParentIfNotMaterialized() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); - expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); expect(parentGetterSupplier.storeNames()).andReturn(new String[]{"store1", "store2"}); replay(parent, parentGetterSupplier); @@ -296,14 +295,14 @@ public void shouldCloseTransformerOnGetterClose() { expect(mockSupplier.get()).andReturn(transformer); expect(parentGetterSupplier.get()).andReturn(parentGetter); - expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); transformer.close(); expectLastCall(); replay(mockSupplier, transformer, parent, parentGetterSupplier); - final KTableValueAndTimestampGetter getter = transformValues.view().get(); + final KTableValueGetter getter = transformValues.view().get(); getter.close(); verify(transformer); @@ -314,7 +313,7 @@ public void shouldCloseParentGetterClose() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, mockSupplier, null); - expect(parent.valueAndTimestampGetterSupplier()).andReturn(parentGetterSupplier); + expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); expect(mockSupplier.get()).andReturn(transformer); expect(parentGetterSupplier.get()).andReturn(parentGetter); @@ -323,7 +322,7 @@ public void shouldCloseParentGetterClose() { replay(mockSupplier, parent, parentGetterSupplier, parentGetter); - final KTableValueAndTimestampGetter getter = transformValues.view().get(); + final KTableValueGetter getter = transformValues.view().get(); getter.close(); verify(parentGetter); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java index 02d3ec9a92b6a..98fb641daf87b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetter; -import org.apache.kafka.streams.kstream.internals.KTableValueAndTimestampGetterSupplier; +import org.apache.kafka.streams.kstream.internals.KTableValueGetter; +import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.processor.api.MockProcessorContext; import org.apache.kafka.streams.processor.api.MockProcessorContext.CapturedForward; import org.apache.kafka.streams.processor.api.Processor; @@ -43,12 +43,13 @@ public class SubscriptionResolverJoinProcessorSupplierTest { private static final ValueJoiner JOINER = (value1, value2) -> "(" + value1 + "," + value2 + ")"; - private static class TestKTableValueGetterSupplier implements KTableValueAndTimestampGetterSupplier { + private static class TestKTableValueGetterSupplier implements + KTableValueGetterSupplier { private final Map map = new HashMap<>(); @Override - public KTableValueAndTimestampGetter get() { - return new KTableValueAndTimestampGetter() { + public KTableValueGetter get() { + return new KTableValueGetter() { @Override public void init(ProcessorContext context) { From 7e8341929ad8ff8d5a3bab8bf54dbc955f7e60cd Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 19:09:19 +0100 Subject: [PATCH 23/40] fix types order --- .../internals/KStreamGlobalKTableJoin.java | 8 ++++---- .../kstream/internals/KStreamImpl.java | 20 +++++++++---------- .../kstream/internals/KStreamImplJoin.java | 20 +++++++++---------- .../kstream/internals/KStreamKStreamJoin.java | 14 ++++++------- .../kstream/internals/KStreamKTableJoin.java | 8 ++++---- .../internals/KStreamKTableJoinProcessor.java | 9 +++++---- .../KStreamSessionWindowAggregate.java | 4 ++-- .../KStreamSlidingWindowAggregate.java | 2 ++ 8 files changed, 44 insertions(+), 41 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java index 0c56a49a1d744..f9d43a663743e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java @@ -21,15 +21,15 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -class KStreamGlobalKTableJoin implements ProcessorSupplier { +class KStreamGlobalKTableJoin implements ProcessorSupplier { private final KTableValueGetterSupplier valueGetterSupplier; - private final ValueJoinerWithKey joiner; + private final ValueJoinerWithKey joiner; private final KeyValueMapper mapper; private final boolean leftJoin; KStreamGlobalKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, - final ValueJoinerWithKey joiner, + final ValueJoinerWithKey joiner, final KeyValueMapper mapper, final boolean leftJoin) { this.valueGetterSupplier = valueGetterSupplier; @@ -39,7 +39,7 @@ class KStreamGlobalKTableJoin implements ProcessorSupplier get() { + public Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), mapper, joiner, leftJoin); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index e13ae57460c41..46d081601f541 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1238,30 +1238,30 @@ private KStream globalTableJoin(final GlobalKTable g } @SuppressWarnings("unchecked") - private KStream doStreamTableJoin(final KTable table, - final ValueJoinerWithKey joiner, - final Joined joined, - final boolean leftJoin) { + private KStream doStreamTableJoin(final KTable table, + final ValueJoinerWithKey joiner, + final Joined joined, + final boolean leftJoin) { Objects.requireNonNull(table, "table can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); - final Set allSourceNodes = ensureCopartitionWith(Collections.singleton((AbstractStream) table)); + final Set allSourceNodes = ensureCopartitionWith(Collections.singleton((AbstractStream) table)); - final JoinedInternal joinedInternal = new JoinedInternal<>(joined); + final JoinedInternal joinedInternal = new JoinedInternal<>(joined); final NamedInternal renamed = new NamedInternal(joinedInternal.name()); final String name = renamed.orElseGenerateWithPrefix(builder, leftJoin ? LEFTJOIN_NAME : JOIN_NAME); - final KStreamKTableJoin processorSupplier = new KStreamKTableJoin<>( - ((KTableImpl) table).valueGetterSupplier(), + final KStreamKTableJoin processorSupplier = new KStreamKTableJoin<>( + ((KTableImpl) table).valueGetterSupplier(), joiner, leftJoin); - final ProcessorParameters processorParameters = new ProcessorParameters<>( + final ProcessorParameters processorParameters = new ProcessorParameters<>( processorSupplier, name); final StreamTableJoinNode streamTableJoinNode = new StreamTableJoinNode<>( name, processorParameters, - ((KTableImpl) table).valueGetterSupplier().storeNames(), + ((KTableImpl) table).valueGetterSupplier().storeNames(), this.name ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java index 77f95de2d2cf5..5af2e9c073c49 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java @@ -53,11 +53,11 @@ class KStreamImplJoin { this.rightOuter = rightOuter; } - public KStream join(final KStream lhs, - final KStream other, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream join(final KStream lhs, + final KStream other, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { final StreamJoinedInternal streamJoinedInternal = new StreamJoinedInternal<>(streamJoined); final NamedInternal renamed = new NamedInternal(streamJoinedInternal.name()); @@ -118,7 +118,7 @@ public KStream join(final KStream lhs, final ProcessorGraphNode otherWindowedStreamsNode = new ProcessorGraphNode<>(otherWindowStreamProcessorName, otherWindowStreamProcessorParams); builder.addGraphNode(otherGraphNode, otherWindowedStreamsNode); - final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>( + final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>( otherWindowStore.name(), windows.beforeMs, windows.afterMs, @@ -126,7 +126,7 @@ public KStream join(final KStream lhs, leftOuter ); - final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>( + final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>( thisWindowStore.name(), windows.afterMs, windows.beforeMs, @@ -134,13 +134,13 @@ public KStream join(final KStream lhs, rightOuter ); - final PassThrough joinMerge = new PassThrough<>(); + final PassThrough joinMerge = new PassThrough<>(); - final StreamStreamJoinNode.StreamStreamJoinNodeBuilder joinBuilder = StreamStreamJoinNode.streamStreamJoinNodeBuilder(); + final StreamStreamJoinNode.StreamStreamJoinNodeBuilder joinBuilder = StreamStreamJoinNode.streamStreamJoinNodeBuilder(); final ProcessorParameters joinThisProcessorParams = new ProcessorParameters<>(joinThis, joinThisName); final ProcessorParameters joinOtherProcessorParams = new ProcessorParameters<>(joinOther, joinOtherName); - final ProcessorParameters joinMergeProcessorParams = new ProcessorParameters<>(joinMerge, joinMergeName); + final ProcessorParameters joinMergeProcessorParams = new ProcessorParameters<>(joinMerge, joinMergeName); joinBuilder.withJoinMergeProcessorParameters(joinMergeProcessorParams) .withJoinThisProcessorParameters(joinThisProcessorParams) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index efea46526588d..df70bc9b2d42f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -32,7 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class KStreamKStreamJoin implements ProcessorSupplier { +class KStreamKStreamJoin implements ProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); @@ -40,13 +40,13 @@ class KStreamKStreamJoin implements ProcessorSupplier private final long joinBeforeMs; private final long joinAfterMs; - private final ValueJoinerWithKey joiner; + private final ValueJoinerWithKey joiner; private final boolean outer; KStreamKStreamJoin(final String otherWindowName, final long joinBeforeMs, final long joinAfterMs, - final ValueJoinerWithKey joiner, + final ValueJoinerWithKey joiner, final boolean outer) { this.otherWindowName = otherWindowName; this.joinBeforeMs = joinBeforeMs; @@ -56,18 +56,18 @@ class KStreamKStreamJoin implements ProcessorSupplier } @Override - public Processor get() { + public Processor get() { return new KStreamKStreamJoinProcessor(); } - private class KStreamKStreamJoinProcessor extends ContextualProcessor { + private class KStreamKStreamJoinProcessor extends ContextualProcessor { private WindowStore otherWindow; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor( @@ -84,7 +84,7 @@ public void process(Record record) { // furthermore, on left/outer joins 'null' in ValueJoiner#apply() indicates a missing record -- // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored if (record.key() == null || record.value() == null) { -// LOG.warn( +//TODO LOG.warn( // "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", // key, value, context().topic(), context().partition(), context().offset() // ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java index 631fad581f7fe..f6f7cc8738254 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java @@ -21,15 +21,15 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -class KStreamKTableJoin implements ProcessorSupplier { +class KStreamKTableJoin implements ProcessorSupplier { private final KeyValueMapper keyValueMapper = (key, value) -> key; private final KTableValueGetterSupplier valueGetterSupplier; - private final ValueJoinerWithKey joiner; + private final ValueJoinerWithKey joiner; private final boolean leftJoin; KStreamKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, - final ValueJoinerWithKey joiner, + final ValueJoinerWithKey joiner, final boolean leftJoin) { this.valueGetterSupplier = valueGetterSupplier; this.joiner = joiner; @@ -37,7 +37,7 @@ class KStreamKTableJoin implements ProcessorSupplier } @Override - public Processor get() { + public Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), keyValueMapper, joiner, leftJoin); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 62d6e75829488..bc9a642ed6f04 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -29,19 +29,19 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KStreamKTableJoinProcessor extends ContextualProcessor { +class KStreamKTableJoinProcessor extends ContextualProcessor { private static final Logger LOG = LoggerFactory.getLogger(KStreamKTableJoin.class); private final KTableValueGetter valueGetter; private final KeyValueMapper keyMapper; - private final ValueJoinerWithKey joiner; + private final ValueJoinerWithKey joiner; private final boolean leftJoin; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; KStreamKTableJoinProcessor(final KTableValueGetter valueGetter, final KeyValueMapper keyMapper, - final ValueJoinerWithKey joiner, + final ValueJoinerWithKey joiner, final boolean leftJoin) { this.valueGetter = valueGetter; this.keyMapper = keyMapper; @@ -50,7 +50,7 @@ class KStreamKTableJoinProcessor extends ContextualProcessor< } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -69,6 +69,7 @@ public void process(final Record record) { // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored final K2 mappedKey = keyMapper.apply(record.key(), record.value()); if (mappedKey == null || record.value() == null) { + //TODO // LOG.warn( // "Skipping record due to null join key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", // key, value, context().topic(), context().partition(), context().offset() diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index e0a702781c3ba..aeea2bbd5f765 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -108,7 +108,7 @@ public void init(final ProcessorContext, Change> context) { public void process(Record record) { // if the key is null, we do not need proceed aggregating // the record with the table - if (record.key() == null) { + if (record.key() == null) { //TODO // LOG.warn( // "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", // value, context().topic(), context().partition(), context().offset() @@ -141,7 +141,7 @@ public void process(Record record) { } } - if (mergedWindow.end() < closeTime) { + if (mergedWindow.end() < closeTime) { //TODO // LOG.warn( // "Skipping record for expired window. " + // "key=[{}] " + diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index 52bb37dfee4d9..7beae84d849ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -109,6 +109,7 @@ public void init(final ProcessorContext, Change> context) { @Override public void process(Record record) { if (record.key() == null || record.value() == null) { + //TODO // log.warn( // "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]", // value, context().topic(), context().partition(), context().offset() @@ -122,6 +123,7 @@ public void process(Record record) { final long closeTime = observedStreamTime - windows.gracePeriodMs(); if (inputRecordTimestamp + 1L + windows.timeDifferenceMs() <= closeTime) { + //TODO // log.warn( // "Skipping record for expired window. " + // "key=[{}] " + From b25fe0006f6a4b23257194f4c335e337c23530c4 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 19:15:45 +0100 Subject: [PATCH 24/40] add todos; --- .../streams/kstream/internals/KStreamWindowAggregate.java | 6 +++--- .../streams/kstream/internals/KTableKTableInnerJoin.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 0d05aa493f03e..9e8f9755183fd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -40,7 +40,7 @@ import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; public class KStreamWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { - private final Logger log = LoggerFactory.getLogger(getClass()); + private final Logger LOG = LoggerFactory.getLogger(getClass()); private final String storeName; private final Windows windows; @@ -105,7 +105,7 @@ public void init(final ProcessorContext, Change> context) { @Override public void process(Record record) { - if (record.key() == null) { + if (record.key() == null) { //TODO // log.warn( // "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", // value, context().topic(), context().partition(), context().offset() @@ -148,7 +148,7 @@ public void process(Record record) { newAgg, sendOldValues ? oldAgg : null, newTimestamp); - } else { + } else { //TODO // log.warn( // "Skipping record for expired window. " + // "key=[{}] " + diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 74d4c728ec74b..4c31a315c3db3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -89,7 +89,7 @@ public void init(final ProcessorContext> context) { @Override public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record - if (record.key() == null) { + if (record.key() == null) { //TODO // LOG.warn( // "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", // change, context().topic(), context().partition(), context().offset() From 71fa2fd29face51aa138d553a40bb61e9db92bd8 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 12 Apr 2021 11:41:08 +0100 Subject: [PATCH 25/40] adjust transformers with adapters --- .../kstream/internals/AbstractStream.java | 71 ++++++++++--------- .../internals/KStreamFlatMapValues.java | 14 ++-- .../internals/KStreamFlatTransform.java | 23 +++--- .../internals/KStreamFlatTransformValues.java | 24 ++++--- .../kstream/internals/KStreamImpl.java | 50 ++++++------- .../internals/KStreamTransformValues.java | 19 ++--- .../internals/KStreamFlatTransformTest.java | 13 ++-- .../KStreamFlatTransformValuesTest.java | 13 ++-- 8 files changed, 114 insertions(+), 113 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java index 2fac9ba69ab3b..c5ab2a8bfc202 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java @@ -21,12 +21,19 @@ import java.util.Objects; import java.util.Set; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.ValueMapperWithKey; +import org.apache.kafka.streams.kstream.ValueTransformer; +import org.apache.kafka.streams.kstream.ValueTransformerSupplier; +import org.apache.kafka.streams.kstream.ValueTransformerWithKey; +import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.graph.GraphNode; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import org.apache.kafka.streams.state.StoreBuilder; /* * Any classes (KTable, KStream, etc) extending this class should follow the serde specification precedence ordering as: @@ -103,38 +110,38 @@ static ValueMapperWithKey withKey(final ValueMapper return (readOnlyKey, value) -> valueMapper.apply(value); } -// static ValueTransformerWithKeySupplier toValueTransformerWithKeySupplier( -// final ValueTransformerSupplier valueTransformerSupplier) { -// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); -// ApiUtils.checkSupplier(valueTransformerSupplier); -// return new ValueTransformerWithKeySupplier() { -// @Override -// public ValueTransformerWithKey get() { -// final ValueTransformer valueTransformer = valueTransformerSupplier.get(); -// return new ValueTransformerWithKey() { -// @Override -// public void init(final ProcessorContext context) { -// valueTransformer.init(context); -// } -// -// @Override -// public VR transform(final K readOnlyKey, final V value) { -// return valueTransformer.transform(value); -// } -// -// @Override -// public void close() { -// valueTransformer.close(); -// } -// }; -// } -// -// @Override -// public Set> stores() { -// return valueTransformerSupplier.stores(); -// } -// }; -// } + static ValueTransformerWithKeySupplier toValueTransformerWithKeySupplier( + final ValueTransformerSupplier valueTransformerSupplier) { + Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); + ApiUtils.checkSupplier(valueTransformerSupplier); + return new ValueTransformerWithKeySupplier() { + @Override + public ValueTransformerWithKey get() { + final ValueTransformer valueTransformer = valueTransformerSupplier.get(); + return new ValueTransformerWithKey() { + @Override + public void init(final ProcessorContext context) { + valueTransformer.init(context); + } + + @Override + public VR transform(final K readOnlyKey, final V value) { + return valueTransformer.transform(value); + } + + @Override + public void close() { + valueTransformer.close(); + } + }; + } + + @Override + public Set> stores() { + return valueTransformerSupplier.stores(); + } + }; + } static ValueJoinerWithKey toValueJoinerWithKey(final ValueJoiner valueJoiner) { Objects.requireNonNull(valueJoiner, "joiner can't be null"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 1008b297b3d3c..2b2fadc5c664e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -17,10 +17,10 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.ProcessorAdapter; class KStreamFlatMapValues implements ProcessorSupplier { @@ -32,15 +32,15 @@ class KStreamFlatMapValues implements ProcessorSupplier get() { - return new KStreamFlatMapValuesProcessor(); + return ProcessorAdapter.adaptRaw(new KStreamFlatMapValuesProcessor()); } - private class KStreamFlatMapValuesProcessor extends ContextualProcessor { + private class KStreamFlatMapValuesProcessor extends AbstractProcessor { @Override - public void process(final Record record) { - final Iterable newValues = mapper.apply(record.key(), record.value()); + public void process(KIn key, VIn value) { + final Iterable newValues = mapper.apply(key, value); for (final VOut v : newValues) { - context().forward(record.withValue(v)); + context().forward(key, v); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java index 783acf557e1b1..ca7bfdcbaa20c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java @@ -19,16 +19,17 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; -import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; -public class KStreamFlatTransform implements ProcessorSupplier { +public class KStreamFlatTransform implements + ProcessorSupplier { private final TransformerSupplier>> transformerSupplier; @@ -38,7 +39,7 @@ public KStreamFlatTransform(final TransformerSupplier get() { - return new KStreamFlatTransformProcessor<>(transformerSupplier.get()); + return ProcessorAdapter.adaptRaw(new KStreamFlatTransformProcessor<>(transformerSupplier.get())); } @Override @@ -47,7 +48,7 @@ public Set> stores() { } public static class KStreamFlatTransformProcessor extends - ContextualProcessor { + AbstractProcessor { private final Transformer>> transformer; @@ -56,18 +57,18 @@ public KStreamFlatTransformProcessor(final Transformer context) { + public void init(final ProcessorContext context) { super.init(context); - //TODO transformer.init(context); + transformer.init(context); } @Override - public void process(Record record) { + public void process(KIn key, VIn value) { final Iterable> pairs = - transformer.transform(record.key(), record.value()); + transformer.transform(key, value); if (pairs != null) { for (final KeyValue pair : pairs) { - context().forward(record.withKey(pair.key).withValue(pair.value)); + context().forward(pair.key, pair.value); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java index 822c89d1bcbfa..d0af6f073eaed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java @@ -18,16 +18,18 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; -public class KStreamFlatTransformValues implements ProcessorSupplier { +public class KStreamFlatTransformValues implements + ProcessorSupplier { private final ValueTransformerWithKeySupplier> valueTransformerSupplier; @@ -37,7 +39,7 @@ public KStreamFlatTransformValues(final ValueTransformerWithKeySupplier get() { - return new KStreamFlatTransformValuesProcessor<>(valueTransformerSupplier.get()); + return ProcessorAdapter.adaptRaw(new KStreamFlatTransformValuesProcessor<>(valueTransformerSupplier.get())); } @Override @@ -46,7 +48,7 @@ public Set> stores() { } public static class KStreamFlatTransformValuesProcessor extends - ContextualProcessor { + AbstractProcessor { private final ValueTransformerWithKey> valueTransformer; @@ -55,18 +57,18 @@ public static class KStreamFlatTransformValuesProcessor extends } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); - // TODO valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } @Override - public void process(Record record) { + public void process(KIn key, VIn value) { final Iterable transformedValues = - valueTransformer.transform(record.key(), record.value()); + valueTransformer.transform(key, value); if (transformedValues != null) { for (final VOut transformedValue : transformedValues) { - context.forward(record.withValue(transformedValue)); + context.forward(key, transformedValue); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 46d081601f541..9feecf0c7139b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1337,27 +1337,23 @@ public KStream flatTransform(final TransformerSupplier KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, final String... stateStoreNames) { - //TODO - return null; -// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); -// return doTransformValues( -// toValueTransformerWithKeySupplier(valueTransformerSupplier), -// NamedInternal.empty(), -// stateStoreNames); + Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); + return doTransformValues( + toValueTransformerWithKeySupplier(valueTransformerSupplier), + NamedInternal.empty(), + stateStoreNames); } @Override public KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, final Named named, final String... stateStoreNames) { - //TODO - return null; -// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); -// Objects.requireNonNull(named, "named can't be null"); -// return doTransformValues( -// toValueTransformerWithKeySupplier(valueTransformerSupplier), -// new NamedInternal(named), -// stateStoreNames); + Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); + Objects.requireNonNull(named, "named can't be null"); + return doTransformValues( + toValueTransformerWithKeySupplier(valueTransformerSupplier), + new NamedInternal(named), + stateStoreNames); } @Override @@ -1408,26 +1404,22 @@ private KStream doTransformValues(final ValueTransformerWithKeySuppl @Override public KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, final String... stateStoreNames) { - //TODO - return null; -// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); -// return doFlatTransformValues( -// toValueTransformerWithKeySupplier(valueTransformerSupplier), -// NamedInternal.empty(), -// stateStoreNames); + Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); + return doFlatTransformValues( + toValueTransformerWithKeySupplier(valueTransformerSupplier), + NamedInternal.empty(), + stateStoreNames); } @Override public KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, final Named named, final String... stateStoreNames) { - //TODO - return null; -// Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); -// return doFlatTransformValues( -// toValueTransformerWithKeySupplier(valueTransformerSupplier), -// named, -// stateStoreNames); + Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); + return doFlatTransformValues( + toValueTransformerWithKeySupplier(valueTransformerSupplier), + named, + stateStoreNames); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index c2f254d5f433c..c43c7e4e91161 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -18,11 +18,12 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; @@ -37,7 +38,7 @@ public class KStreamTransformValues implements ProcessorSupplier get() { - return new KStreamTransformValuesProcessor<>(valueTransformerSupplier.get()); + return ProcessorAdapter.adaptRaw(new KStreamTransformValuesProcessor<>(valueTransformerSupplier.get())); } @Override @@ -46,7 +47,7 @@ public Set> stores() { } public static class KStreamTransformValuesProcessor extends - ContextualProcessor { + AbstractProcessor { private final ValueTransformerWithKey valueTransformer; @@ -55,14 +56,14 @@ public static class KStreamTransformValuesProcessor extends } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); - //TODO valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } @Override - public void process(Record record) { - context.forward(record.withValue(valueTransformer.transform(record.key(), record.value()))); + public void process(K key, V value) { + context.forward(key, valueTransformer.transform(key, value)); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java index 5dc382cea6817..4c200a134bc24 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.kstream.internals.KStreamFlatTransform.KStreamFlatTransformProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.api.Record; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.junit.Before; @@ -38,7 +37,7 @@ public class KStreamFlatTransformTest extends EasyMockSupport { private Number inputValue; private Transformer>> transformer; - private ProcessorContext context; + private org.apache.kafka.streams.processor.ProcessorContext context; private KStreamFlatTransformProcessor processor; @@ -53,7 +52,7 @@ public void setUp() { @Test public void shouldInitialiseFlatTransformProcessor() { -// TODO transformer.init(context); + transformer.init(context); replayAll(); processor.init(context); @@ -72,11 +71,11 @@ public void shouldTransformInputRecordToMultipleOutputRecords() { EasyMock.expect(transformer.transform(inputKey, inputValue)).andReturn(outputRecords); for (final KeyValue outputRecord : outputRecords) { - context.forward(new Record<>(outputRecord.key, outputRecord.value, 0L)); + context.forward(outputRecord.key, outputRecord.value); } replayAll(); - processor.process(new Record<>(inputKey, inputValue, 0L)); + processor.process(inputKey, inputValue); verifyAll(); } @@ -90,7 +89,7 @@ public void shouldAllowEmptyListAsResultOfTransform() { .andReturn(Collections.emptyList()); replayAll(); - processor.process(new Record<>(inputKey, inputValue, 0L)); + processor.process(inputKey, inputValue); verifyAll(); } @@ -104,7 +103,7 @@ public void shouldAllowNullAsResultOfTransform() { .andReturn(null); replayAll(); - processor.process(new Record<>(inputKey, inputValue, 0L)); + processor.process(inputKey, inputValue); verifyAll(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java index 957f2c11a4aab..d0266309b2606 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java @@ -24,9 +24,8 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.KStreamFlatTransformValues.KStreamFlatTransformValuesProcessor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; @@ -39,7 +38,7 @@ public class KStreamFlatTransformValuesTest extends EasyMockSupport { private Integer inputValue; private ValueTransformerWithKey> valueTransformer; - private ProcessorContext context; + private ProcessorContext context; private KStreamFlatTransformValuesProcessor processor; @@ -73,11 +72,11 @@ public void shouldTransformInputRecordToMultipleOutputValues() { EasyMock.expect(valueTransformer.transform(inputKey, inputValue)).andReturn(outputValues); for (final String outputValue : outputValues) { - context.forward(new Record<>(inputKey, outputValue, 0)); + context.forward(inputKey, outputValue); } replayAll(); - processor.process(new Record<>(inputKey, inputValue, 0L)); + processor.process(inputKey, inputValue); verifyAll(); } @@ -90,7 +89,7 @@ public void shouldEmitNoRecordIfTransformReturnsEmptyList() { EasyMock.expect(valueTransformer.transform(inputKey, inputValue)).andReturn(Collections.emptyList()); replayAll(); - processor.process(new Record<>(inputKey, inputValue, 0L)); + processor.process(inputKey, inputValue); verifyAll(); } @@ -103,7 +102,7 @@ public void shouldEmitNoRecordIfTransformReturnsNull() { EasyMock.expect(valueTransformer.transform(inputKey, inputValue)).andReturn(null); replayAll(); - processor.process(new Record<>(inputKey, inputValue, 0L)); + processor.process(inputKey, inputValue); verifyAll(); } From 32f05a12b93b4dee1e5a92c3c009118c3b080cde Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 12 Apr 2021 12:27:54 +0100 Subject: [PATCH 26/40] checkstail main --- .../kstream/internals/KStreamAggregate.java | 19 ++++-- .../internals/KStreamFlatMapValues.java | 2 +- .../internals/KStreamFlatTransform.java | 2 +- .../internals/KStreamFlatTransformValues.java | 2 +- .../kstream/internals/KStreamImpl.java | 2 +- .../kstream/internals/KStreamJoinWindow.java | 2 +- .../kstream/internals/KStreamKStreamJoin.java | 14 ++-- .../kstream/internals/KStreamReduce.java | 19 ++++-- .../KStreamSessionWindowAggregate.java | 64 ++++++++++-------- .../KStreamSlidingWindowAggregate.java | 4 +- .../internals/KStreamTransformValues.java | 2 +- .../internals/KStreamWindowAggregate.java | 66 ++++++++++--------- .../kstream/internals/KTableAggregate.java | 2 +- .../kstream/internals/KTableFilter.java | 2 +- .../internals/KTableKTableInnerJoin.java | 16 +++-- .../internals/KTableKTableJoinMerger.java | 2 +- .../internals/KTableKTableLeftJoin.java | 16 +++-- .../internals/KTableKTableOuterJoin.java | 14 ++-- .../internals/KTableKTableRightJoin.java | 16 +++-- .../kstream/internals/KTableMapValues.java | 2 +- ...KTableMaterializedValueGetterSupplier.java | 2 +- .../kstream/internals/KTableReduce.java | 2 +- .../internals/KTableRepartitionMap.java | 4 +- .../kstream/internals/KTableSource.java | 12 ++-- .../KTableSourceValueGetterSupplier.java | 2 +- .../internals/KTableTransformValues.java | 4 +- .../TupleChangeCacheFlushListener.java | 2 +- ...eignJoinSubscriptionProcessorSupplier.java | 14 ++-- ...JoinSubscriptionSendProcessorSupplier.java | 2 +- ...scriptionJoinForeignProcessorSupplier.java | 2 +- ...criptionResolverJoinProcessorSupplier.java | 2 +- ...criptionStoreReceiveProcessorSupplier.java | 2 +- .../KTableSuppressProcessorSupplier.java | 4 +- .../internals/InternalTopologyBuilder.java | 2 +- .../streams/state/ValueAndTimestamp.java | 2 +- ...caughtExceptionHandlerIntegrationTest.java | 2 +- .../kstream/RepartitionTopicNamingTest.java | 2 +- .../kstream/internals/AbstractStreamTest.java | 2 +- ...tionResolverJoinProcessorSupplierTest.java | 2 +- .../graph/GraphGraceSearchUtilTest.java | 4 +- .../graph/TableProcessorNodeTest.java | 2 +- .../internals/ProcessorNodeTest.java | 4 +- .../internals/RepartitionOptimizingTest.java | 2 +- .../processor/internals/StreamThreadTest.java | 2 +- .../kafka/streams/tests/SmokeTestUtil.java | 2 +- 45 files changed, 199 insertions(+), 151 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index b13c46ef7da96..cebd4bce86445 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -19,9 +19,11 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -58,7 +60,7 @@ public void enableSendingOldValues() { } - private class KStreamAggregateProcessor implements Processor> { + private class KStreamAggregateProcessor extends ContextualProcessor> { private TimestampedKeyValueStore store; private Sensor droppedRecordsSensor; private TupleChangeForwarder tupleForwarder; @@ -78,13 +80,16 @@ public void init(final ProcessorContext> context) { } @Override - public void process(Record record) { + public void process(final Record record) { // If the key or value is null we don't need to proceed if (record.key() == null || record.value() == null) { -// LOG.warn( -// "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// key, value, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.key(), record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -131,7 +136,7 @@ private class KStreamAggregateValueGetter implements KTableValueGetter { @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 2b2fadc5c664e..9197103c9e362 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -37,7 +37,7 @@ public Processor get() { private class KStreamFlatMapValuesProcessor extends AbstractProcessor { @Override - public void process(KIn key, VIn value) { + public void process(final KIn key, final VIn value) { final Iterable newValues = mapper.apply(key, value); for (final VOut v : newValues) { context().forward(key, v); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java index ca7bfdcbaa20c..7b1133807a42a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java @@ -63,7 +63,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(KIn key, VIn value) { + public void process(final KIn key, final VIn value) { final Iterable> pairs = transformer.transform(key, value); if (pairs != null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java index d0af6f073eaed..ba129a7b40949 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java @@ -63,7 +63,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(KIn key, VIn value) { + public void process(final KIn key, final VIn value) { final Iterable transformedValues = valueTransformer.transform(key, value); if (transformedValues != null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 9feecf0c7139b..950d2671863db 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -748,7 +748,7 @@ public KTable toTable(final Named named, subTopologySourceNodes = this.subTopologySourceNodes; } - ProcessorSupplier> tableSource = new KTableSource<>( + final ProcessorSupplier> tableSource = new KTableSource<>( materializedInternal.storeName(), materializedInternal.queryableStoreName() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index c76b0cc168335..0ed2d6919d89b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -49,7 +49,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(Record record) { + public void process(final Record record) { // if the key is null, we do not need to put the record into window store // since it will never be considered for join operations if (record.key() != null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index df70bc9b2d42f..12aadba70b5cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; @@ -76,7 +77,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(Record record) { + public void process(final Record record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record // // we also ignore the record if value is null, because in a key-value data model a null-value indicates @@ -84,10 +85,13 @@ public void process(Record record) { // furthermore, on left/outer joins 'null' in ValueJoiner#apply() indicates a missing record -- // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored if (record.key() == null || record.value() == null) { -//TODO LOG.warn( -// "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// key, value, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.key(), record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index afb97c83c7e8c..52929cde30e2b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -18,9 +18,11 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.Reducer; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -54,7 +56,7 @@ public void enableSendingOldValues() { } - private class KStreamReduceProcessor implements Processor> { + private class KStreamReduceProcessor extends ContextualProcessor> { private TimestampedKeyValueStore store; private TupleChangeForwarder tupleForwarder; private StreamsMetricsImpl metrics; @@ -73,13 +75,16 @@ public void init(final ProcessorContext> context) { } @Override - public void process(Record record) { + public void process(final Record record) { // If the key or value is null we don't need to proceed if (record.key() == null || record.value() == null) { -// LOG.warn( -// "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// key, value, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.key(), record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -123,7 +128,7 @@ private class KStreamReduceValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index aeea2bbd5f765..683547237b4ee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -24,9 +24,11 @@ import org.apache.kafka.streams.kstream.Merger; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; @@ -78,7 +80,8 @@ public void enableSendingOldValues() { sendOldValues = true; } - private class KStreamSessionWindowAggregateProcessor implements Processor, Change> { + private class KStreamSessionWindowAggregateProcessor extends + ContextualProcessor, Change> { private SessionStore store; private SessionRecordForwarder tupleForwarder; @@ -89,7 +92,7 @@ private class KStreamSessionWindowAggregateProcessor implements Processor, Change> context) { - InternalProcessorContext, Change> internalProcessorContext = + final InternalProcessorContext, Change> internalProcessorContext = (InternalProcessorContext, Change>) context; metrics = (StreamsMetricsImpl) context.metrics(); final String threadId = Thread.currentThread().getName(); @@ -105,14 +108,17 @@ public void init(final ProcessorContext, Change> context) { } @Override - public void process(Record record) { + public void process(final Record record) { // if the key is null, we do not need proceed aggregating // the record with the table - if (record.key() == null) { //TODO -// LOG.warn( -// "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// value, context().topic(), context().partition(), context().offset() -// ); + if (record.key() == null) { + LOG.warn( + "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -142,26 +148,26 @@ public void process(Record record) { } if (mergedWindow.end() < closeTime) { //TODO -// LOG.warn( -// "Skipping record for expired window. " + -// "key=[{}] " + -// "topic=[{}] " + -// "partition=[{}] " + -// "offset=[{}] " + -// "timestamp=[{}] " + -// "window=[{},{}] " + -// "expiration=[{}] " + -// "streamTime=[{}]", -// key, -// context().topic(), -// context().partition(), -// context().offset(), -// timestamp, -// mergedWindow.start(), -// mergedWindow.end(), -// closeTime, -// observedStreamTime -// ); + LOG.warn( + "Skipping record for expired window. " + + "key=[{}] " + + "topic=[{}] " + + "partition=[{}] " + + "offset=[{}] " + + "timestamp=[{}] " + + "window=[{},{}] " + + "expiration=[{}] " + + "streamTime=[{}]", + record.key(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L), + timestamp, + mergedWindow.start(), + mergedWindow.end(), + closeTime, + observedStreamTime + ); lateRecordDropSensor.record(); } else { if (!mergedWindow.equals(newSessionWindow)) { @@ -205,7 +211,7 @@ private class KTableSessionWindowValueGetter private SessionStore store; @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index 7beae84d849ba..3a51510d9cfd9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -107,7 +107,7 @@ public void init(final ProcessorContext, Change> context) { @Override - public void process(Record record) { + public void process(final Record record) { if (record.key() == null || record.value() == null) { //TODO // log.warn( @@ -526,7 +526,7 @@ private class KStreamWindowAggregateValueGetter @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { windowStore = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index c43c7e4e91161..1d028e10105a4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -62,7 +62,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(K key, V value) { + public void process(final K key, final V value) { context.forward(key, valueTransformer.transform(key, value)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 9e8f9755183fd..8cbfbc6141a25 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -23,9 +23,11 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windows; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedWindowStore; @@ -40,7 +42,7 @@ import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; public class KStreamWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { - private final Logger LOG = LoggerFactory.getLogger(getClass()); + private final static Logger LOG = LoggerFactory.getLogger(KStreamWindowAggregate.class); private final String storeName; private final Windows windows; @@ -74,18 +76,19 @@ public void enableSendingOldValues() { } - private class KStreamWindowAggregateProcessor implements Processor, Change> { + private class KStreamWindowAggregateProcessor extends + ContextualProcessor, Change> { private TimestampedWindowStore windowStore; private TupleChangeForwarder, Agg> tupleForwarder; private StreamsMetricsImpl metrics; - private InternalProcessorContext internalProcessorContext; private Sensor lateRecordDropSensor; private Sensor droppedRecordsSensor; private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; @Override public void init(final ProcessorContext, Change> context) { - internalProcessorContext = (InternalProcessorContext) context; + final InternalProcessorContext, Change> internalProcessorContext = + (InternalProcessorContext, Change>) context; metrics = internalProcessorContext.metrics(); final String threadId = Thread.currentThread().getName(); lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor( @@ -104,12 +107,15 @@ public void init(final ProcessorContext, Change> context) { } @Override - public void process(Record record) { + public void process(final Record record) { if (record.key() == null) { //TODO -// log.warn( -// "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// value, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -148,26 +154,26 @@ public void process(Record record) { newAgg, sendOldValues ? oldAgg : null, newTimestamp); - } else { //TODO -// log.warn( -// "Skipping record for expired window. " + -// "key=[{}] " + -// "topic=[{}] " + -// "partition=[{}] " + -// "offset=[{}] " + -// "timestamp=[{}] " + -// "window=[{},{}) " + -// "expiration=[{}] " + -// "streamTime=[{}]", -// key, -// context().topic(), -// context().partition(), -// context().offset(), -// context().timestamp(), -// windowStart, windowEnd, -// closeTime, -// observedStreamTime -// ); + } else { + LOG.warn( + "Skipping record for expired window. " + + "key=[{}] " + + "topic=[{}] " + + "partition=[{}] " + + "offset=[{}] " + + "timestamp=[{}] " + + "window=[{},{}) " + + "expiration=[{}] " + + "streamTime=[{}]", + record.key(), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L), + record.timestamp(), + windowStart, windowEnd, + closeTime, + observedStreamTime + ); lateRecordDropSensor.record(); } } @@ -195,7 +201,7 @@ private class KStreamWindowAggregateValueGetter implements KTableValueGetter void init(ProcessorContext context) { + public void init(final ProcessorContext context) { windowStore = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index 31c9e6b6fb616..89d1cd5d46716 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -76,7 +76,7 @@ public void init(final ProcessorContext> context) { * @throws StreamsException if key is null */ @Override - public void process(Record> record) { + public void process(final Record> record) { // the keys should never be null if (record.key() == null) { throw new StreamsException("Record key for KTable aggregate operator with state " + storeName + " should not be null."); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java index 925bb06ba2600..c9ddb48e89b22 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java @@ -164,7 +164,7 @@ private class KTableFilterValueGetter implements KTableValueGetter { } @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { parentGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 4c31a315c3db3..e0fce0bfd08bc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; @@ -89,11 +90,14 @@ public void init(final ProcessorContext> context) { @Override public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record - if (record.key() == null) { //TODO -// LOG.warn( -// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", -// change, context().topic(), context().partition(), context().offset() -// ); + if (record.key() == null) { + LOG.warn( + "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -140,7 +144,7 @@ private class KTableKTableInnerJoinValueGetter implements KTableValueGetter void init(ProcessorContext context) { + public void init(final ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java index 4db2167d057b8..b9152caf4b279 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java @@ -115,7 +115,7 @@ public void init(final ProcessorContext> context) { } @Override - public void process(Record> record) { + public void process(final Record> record) { if (queryableName != null) { store.put(record.key(), ValueAndTimestamp.make(record.value().newValue, record.timestamp())); tupleForwarder.maybeForward(record, record.value().newValue, sendOldValues ? record.value().oldValue : null); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 9b35a7cee2c24..dafe4f46121c0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; @@ -82,13 +83,16 @@ public void init(final ProcessorContext> context) { } @Override - public void process(Record> record) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record if (record.key() == null) { -// LOG.warn( -// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", -// change, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -142,7 +146,7 @@ private class KTableKTableLeftJoinValueGetter implements KTableValueGetter } @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index f0a41e7378a1d..86c921e3bd7e9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; @@ -84,10 +85,13 @@ public void init(final ProcessorContext> context) { public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record if (record.key() == null) { -// LOG.warn( -// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", -// change, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -137,7 +141,7 @@ private class KTableKTableOuterJoinValueGetter implements KTableValueGetter void init(ProcessorContext context) { + public void init(final ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index c583d5a1460c7..93e7d60a32f72 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; @@ -80,13 +81,16 @@ public void init(final ProcessorContext> context) { } @Override - public void process(Record> record) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record if (record.key() == null) { -// LOG.warn( -// "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", -// change, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -132,7 +136,7 @@ private class KTableKTableRightJoinValueGetter implements KTableValueGetter void init(ProcessorContext context) { + public void init(final ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java index 350ab8cc27df4..f1ebb52975240 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java @@ -155,7 +155,7 @@ private class KTableMapValuesValueGetter implements KTableValueGetter { } @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { parentGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java index fa085717e5a04..db682163746e3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java @@ -41,7 +41,7 @@ private class KTableMaterializedValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index b3126881f0ecb..323060dc11eec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -71,7 +71,7 @@ public void init(final ProcessorContext> context) { * @throws StreamsException if key is null */ @Override - public void process(Record> record) { + public void process(final Record> record) { // the keys should never be null if (record.key() == null) { throw new StreamsException("Record key for KTable reduce operator with state " + storeName + " should not be null."); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index 5ecfffec9976b..73e6a5e745bec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -85,7 +85,7 @@ private class KTableMapProcessor extends ContextualProcessor, K1, C * @throws StreamsException if key is null */ @Override - public void process(Record> record) { + public void process(final Record> record) { // the original key should never be null if (record.key() == null) { throw new StreamsException( @@ -123,7 +123,7 @@ private class KTableMapValueGetter implements KTableValueGetter void init(ProcessorContext context) { + public void init(final ProcessorContext context) { parentGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index 56cc6fca95977..29dca4012a776 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -99,11 +100,12 @@ public void init(final ProcessorContext> context) { public void process(final Record record) { // if the key is null, then ignore the record if (record.key() == null) { - // TODO check if context needed for logging -// LOG.warn( -// "Skipping record due to null key. topic=[{}] partition=[{}] offset=[{}]", -// context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key. topic=[{}] partition=[{}] offset=[{}]", + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java index 337add01e6f1d..cb6b28de3f463 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java @@ -41,7 +41,7 @@ private class KTableSourceValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store = null; @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java index 27fa802854a6d..ce648f33cd71c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java @@ -110,7 +110,7 @@ public void init(final ProcessorContext> context) { } @Override - public void process(Record> record) { + public void process(final Record> record) { final V1 newValue = valueTransformer.transform(record.key(), record.value().newValue); if (queryableName == null) { @@ -141,7 +141,7 @@ private class KTableTransformValuesGetter implements KTableValueGetter { } @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { parentGetter.init(context); // valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java index 241efd3158fff..25de66d8eac4e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleChangeCacheFlushListener.java @@ -43,7 +43,7 @@ public void apply(final K key, final ProcessorNode prev = context.currentNode(); context.setCurrentNode(myNode); try { - Record> record = new Record<>( + final Record> record = new Record<>( key, new Change<>(getValueOrNull(newValue), getValueOrNull(oldValue)), newValue != null ? newValue.timestamp() : timestamp); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java index cd950fe8e0e45..bc0328d8cc1ab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java @@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.KeyValueIterator; @@ -82,14 +83,17 @@ public void init(final ProcessorContext> cont * @throws StreamsException if key is null */ @Override - public void process(Record> record) { + public void process(final Record> record) { // if the key is null, we do not need proceed aggregating // the record with the table if (record.key() == null) { -// LOG.warn( -// "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// value, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java index f33d482a46527..0a13453b878f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java @@ -97,7 +97,7 @@ public void init(final ProcessorContext> context) { } @Override - public void process(Record> record) { + public void process(final Record> record) { final long[] currentHash = record.value().newValue == null ? null : Murmur3.hash128(valueSerializer.serialize(valueSerdeTopic, record.value().newValue)); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java index 9da1ca373f91b..f7798ef0cf5d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java @@ -66,7 +66,7 @@ public void init( @Override public void process( - Record, Change>>> record) { + final Record, Change>>> record) { Objects.requireNonNull(record.key(), "This processor should never see a null key."); Objects.requireNonNull(record.value(), "This processor should never see a null value."); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java index e0e9c10fbec97..583143831ee17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java @@ -81,7 +81,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(Record> record) { + public void process(final Record> record) { if (record.value().getVersion() != SubscriptionResponseWrapper.CURRENT_VERSION) { //Guard against modifications to SubscriptionResponseWrapper. Need to ensure that there is //compatibility with previous versions to enable rolling upgrades. Must develop a strategy for diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java index fe17caad2ce3c..7be67a0617075 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java @@ -74,7 +74,7 @@ public void init(final ProcessorContext, Change> record) { + public void process(final Record> record) { if (record.key() == null) { // LOG.warn( // "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java index ebbb79b695b6c..f93d138629d6e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java @@ -73,7 +73,7 @@ public KTableValueGetter get() { private TimeOrderedKeyValueBuffer buffer; @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { parentGetter.init(context); // the main processor is responsible for the buffer's lifecycle buffer = requireNonNull(context.getStateStore(storeName)); @@ -211,7 +211,7 @@ private boolean shouldForward(final Change value) { } @Override - public void process(Record> record) { + public void process(final Record> record) { observedStreamTime = Math.max(observedStreamTime, internalProcessorContext.timestamp()); buffer(record.key(), record.value()); enforceConstraints(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 78cb1dcd65f1b..7c7fae6f82db6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -740,7 +740,7 @@ private void connectProcessorAndStateStore(final String processorName, if (nodeFactory instanceof SourceNodeFactory) { sourceNodes.add((SourceNodeFactory) nodeFactory); } else if (nodeFactory instanceof ProcessorNodeFactory) { - sourceNodes.addAll(findSourcesForProcessorPredecessors((nodeFactory).predecessors)); + sourceNodes.addAll(findSourcesForProcessorPredecessors(nodeFactory.predecessors)); } } return sourceNodes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java index 5ce42e8a0cc20..9874b7c312f35 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java @@ -63,7 +63,7 @@ public static V getValueOrNull(final ValueAndTimestamp valueAndTimestamp) return valueAndTimestamp == null ? null : valueAndTimestamp.value(); } - public static ValueAndTimestamp make(Record record) { + public static ValueAndTimestamp make(final Record record) { return record.value() == null ? null : new ValueAndTimestamp<>(record.value(), record.timestamp()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index f037a230f64a5..19d860a4552cc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -234,7 +234,7 @@ private static class ShutdownProcessor extends ContextualProcessor record) { + public void process(final Record record) { valueList.add(record + " " + context.taskId()); if (throwError.get()) { throw new StreamsException(Thread.currentThread().getName()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java index c1ed533271715..3c251d1aaefd7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java @@ -515,7 +515,7 @@ private static class SimpleProcessor implements Processor record) { + public void process(final Record record) { valueList.add(record.value()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java index 8b78ca7e91c6b..fedf98e998ca6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java @@ -123,7 +123,7 @@ public Processor get() { private class ExtendedKStreamDummyProcessor extends ContextualProcessor { @Override - public void process(Record record) { + public void process(final Record record) { // flip a coin and filter if (rand.nextBoolean()) { context().forward(record); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java index 98fb641daf87b..8a9bad4858969 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java @@ -52,7 +52,7 @@ public KTableValueGetter get() { return new KTableValueGetter() { @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java index 712ebb77c6556..285cb5bb8e584 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java @@ -56,7 +56,7 @@ public void shouldFailIfThereIsNoGraceAncestor() { public void init(final ProcessorContext context) {} @Override - public void process(Record record) { + public void process(final Record record) { } @Override @@ -141,7 +141,7 @@ public void shouldExtractGraceFromSessionAncestorThroughStatefulParent() { public void init(final ProcessorContext context) {} @Override - public void process(Record record) { + public void process(final Record record) { } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java index 3488a96fa4031..2cad5965e9778 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java @@ -27,7 +27,7 @@ public class TableProcessorNodeTest { private static class TestProcessor implements Processor { @Override - public void process(Record record) { + public void process(final Record record) { } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 4eba431442a25..0207c1b2f800f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -69,7 +69,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(Record record) { + public void process(final Record record) { throw new RuntimeException(); } @@ -85,7 +85,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(Record record) { + public void process(final Record record) { } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java index e142f8ffb5215..767a075330c90 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java @@ -267,7 +267,7 @@ private static class SimpleProcessor implements Processor record) { + public void process(final Record record) { valueList.add(record.value()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 99e3c2909e3b1..2123efea6d4de 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -1815,7 +1815,7 @@ public void shouldPunctuateActiveTask() { final ProcessorSupplier punctuateProcessor = () -> new ContextualProcessor() { @Override - public void process(Record record) { + public void process(final Record record) { } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index afc5992810ed8..6f63fd06b3fd8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -55,7 +55,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(Record record) { + public void process(final Record record) { numRecordsProcessed++; if (numRecordsProcessed % 100 == 0) { System.out.printf("%s: %s%n", name, Instant.now()); From 7c18abd91afb9e65f7b738a5a63f1d772883a6f9 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 12 Apr 2021 12:31:16 +0100 Subject: [PATCH 27/40] checkstyle test --- ...caughtExceptionHandlerIntegrationTest.java | 47 +++++++++---------- .../internals/KStreamTransformValuesTest.java | 16 ++----- .../internals/KStreamWindowAggregateTest.java | 2 +- .../internals/KTableKTableOuterJoinTest.java | 2 +- .../internals/KTableMapValuesTest.java | 2 +- ...tionResolverJoinProcessorSupplierTest.java | 2 +- .../suppress/KTableSuppressProcessorTest.java | 1 - .../kafka/streams/tests/SmokeTestUtil.java | 4 +- 8 files changed, 32 insertions(+), 44 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index 19d860a4552cc..ae020950d9e40 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -17,6 +17,28 @@ package org.apache.kafka.streams.integration; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; +import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.REPLACE_THREAD; +import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION; +import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KafkaStreams; @@ -30,9 +52,7 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Named; -import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.api.ContextualProcessor; -import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; @@ -48,29 +68,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.common.utils.Utils.mkObjectProperties; -import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.REPLACE_THREAD; -import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION; -import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.fail; - @Category(IntegrationTest.class) @SuppressWarnings("deprecation") //Need to call the old handler, will remove those calls when the old handler is removed public class StreamsUncaughtExceptionHandlerIntegrationTest { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index 56ab2c6c2b009..eaf718f7b9644 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -16,23 +16,23 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertArrayEquals; + +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValueTimestamp; -import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.ValueTransformer; import org.apache.kafka.streams.kstream.ValueTransformerSupplier; import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; -import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.NoOpValueTransformerWithKeySupplier; import org.apache.kafka.test.StreamsTestUtils; import org.easymock.EasyMockRunner; import org.easymock.Mock; @@ -40,12 +40,6 @@ import org.junit.Test; import org.junit.runner.RunWith; -import java.util.Properties; - -import static org.hamcrest.CoreMatchers.isA; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertArrayEquals; - @RunWith(EasyMockRunner.class) public class KStreamTransformValuesTest { private final String topicName = "topic"; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 8b771d15603e0..c1f9d656968ef 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -180,7 +180,7 @@ public void testJoin() { inputTopic1.pipeInput("D", "4", 3L); inputTopic1.pipeInput("A", "1", 9L); - List, String, ?, ?>> processors = supplier.capturedProcessors(3); + final List, String, ?, ?>> processors = supplier.capturedProcessors(3); processors.get(0).checkAndClearProcessResult( new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(0, 10)), "0+1", 0), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 6c07db130cbf7..3a6343606a129 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -419,7 +419,7 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKeyWithBuiltInMetricsVer private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtInMetricsVersion) { final StreamsBuilder builder = new StreamsBuilder(); - Processor, String, Change> join = new KTableKTableOuterJoin<>( + final Processor, String, Change> join = new KTableKTableOuterJoin<>( (KTableImpl) builder .table("left", Consumed.with(Serdes.String(), Serdes.String())), (KTableImpl) builder diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 8fee802e58405..b726ccab7dd6d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -56,7 +56,7 @@ public class KTableMapValuesTest { private void doTestKTable(final StreamsBuilder builder, final String topic1, - final MockProcessorSupplier supplier) { + final MockProcessorSupplier supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java index 8a9bad4858969..66126f9ab2657 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java @@ -145,7 +145,7 @@ public void shouldForwardWhenHashMatches() { valueGetterSupplier.put("lhs1", "lhsValue"); final long[] hash = Murmur3.hash128(STRING_SERIALIZER.serialize("topic-join-resolver", "lhsValue")); processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue"), 0)); - List> forwarded = context.forwarded(); + final List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); assertThat(forwarded.get(0).record().key(), is("lhs1")); assertThat(forwarded.get(0).record().value(), is("(lhsValue,rhsValue)")); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java index 356721d035bd7..ec75fbcf89011 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java @@ -48,7 +48,6 @@ import org.apache.kafka.streams.processor.MockProcessorContext.CapturedForward; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.api.MockProcessorContext; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.ProcessorNode; diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index 6f63fd06b3fd8..86989526f1fa1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.tests; +import java.time.Instant; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; @@ -24,11 +25,8 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.api.ContextualProcessor; -import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; - -import java.time.Instant; import org.apache.kafka.streams.processor.api.Record; public class SmokeTestUtil { From 34336fef3169512edc1791d7724f1fce5ecf6341 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 12 Apr 2021 15:59:10 +0100 Subject: [PATCH 28/40] passing tests --- .../kstream/internals/KStreamAggregate.java | 7 +- .../internals/KStreamFlatTransform.java | 6 +- .../internals/KStreamFlatTransformValues.java | 6 +- .../kstream/internals/KStreamJoinWindow.java | 6 +- .../kstream/internals/KStreamKStreamJoin.java | 6 +- .../internals/KStreamKTableJoinProcessor.java | 19 ++-- .../kstream/internals/KStreamReduce.java | 7 +- .../KStreamSessionWindowAggregate.java | 9 +- .../KStreamSlidingWindowAggregate.java | 98 ++++++++++--------- .../internals/KStreamWindowAggregate.java | 7 +- .../internals/KTableKTableInnerJoin.java | 6 +- .../internals/KTableKTableLeftJoin.java | 6 +- .../internals/KTableKTableOuterJoin.java | 6 +- .../internals/KTableKTableRightJoin.java | 6 +- .../kstream/internals/KTableSource.java | 6 +- .../internals/SessionCacheFlushListener.java | 4 +- ...eignJoinSubscriptionProcessorSupplier.java | 6 +- ...JoinSubscriptionSendProcessorSupplier.java | 34 ++++--- ...scriptionJoinForeignProcessorSupplier.java | 25 +++-- ...criptionStoreReceiveProcessorSupplier.java | 20 ++-- .../internals/KStreamFlatTransformTest.java | 6 +- .../KStreamFlatTransformValuesTest.java | 4 +- ...amSessionWindowAggregateProcessorTest.java | 20 ++-- .../internals/KTableKTableInnerJoinTest.java | 2 +- .../internals/KTableKTableOuterJoinTest.java | 2 +- .../internals/KTableKTableRightJoinTest.java | 2 +- .../SessionCacheFlushListenerTest.java | 20 ++-- .../TimestampedCacheFlushListenerTest.java | 28 +++--- ...tionResolverJoinProcessorSupplierTest.java | 16 +-- .../kafka/streams/TopologyTestDriverTest.java | 4 +- 30 files changed, 213 insertions(+), 181 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index cebd4bce86445..6780a51680271 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -67,6 +67,7 @@ private class KStreamAggregateProcessor extends ContextualProcessor> context) { + super.init(context); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), context.taskId().toString(), @@ -86,9 +87,9 @@ public void process(final Record record) { LOG.warn( "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", record.key(), record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java index 7b1133807a42a..650c745515519 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java @@ -39,7 +39,11 @@ public KStreamFlatTransform(final TransformerSupplier get() { - return ProcessorAdapter.adaptRaw(new KStreamFlatTransformProcessor<>(transformerSupplier.get())); + return ProcessorAdapter.adaptRaw(getInternal()); + } + + org.apache.kafka.streams.processor.Processor getInternal() { + return new KStreamFlatTransformProcessor<>(transformerSupplier.get()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java index ba129a7b40949..3693b747a9cee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java @@ -39,7 +39,11 @@ public KStreamFlatTransformValues(final ValueTransformerWithKeySupplier get() { - return ProcessorAdapter.adaptRaw(new KStreamFlatTransformValuesProcessor<>(valueTransformerSupplier.get())); + return ProcessorAdapter.adaptRaw(getInternal()); + } + + org.apache.kafka.streams.processor.Processor getInternal() { + return new KStreamFlatTransformValuesProcessor<>(valueTransformerSupplier.get()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index 0ed2d6919d89b..4418bb38a1245 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -40,12 +40,10 @@ private class KStreamJoinWindowProcessor extends ContextualProcessor private WindowStore window; - @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); - - window = (WindowStore) context.getStateStore(windowName); + window = context.getStateStore(windowName); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 12aadba70b5cf..ee8c1d8f642fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -88,9 +88,9 @@ public void process(final Record record) { LOG.warn( "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", record.key(), record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index bc9a642ed6f04..e1329eac1ade6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -16,19 +16,20 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; +import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; + import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; -import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; - class KStreamKTableJoinProcessor extends ContextualProcessor { private static final Logger LOG = LoggerFactory.getLogger(KStreamKTableJoin.class); @@ -69,11 +70,13 @@ public void process(final Record record) { // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored final K2 mappedKey = keyMapper.apply(record.key(), record.value()); if (mappedKey == null || record.value() == null) { - //TODO -// LOG.warn( -// "Skipping record due to null join key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// key, value, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null join key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.key(), record.value(), + context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context.recordMetadata().map(RecordMetadata::partition).orElse(-1), + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); } else { final V2 value2 = getValueOrNull(valueGetter.get(mappedKey)); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index 52929cde30e2b..6176bea916bef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -64,6 +64,7 @@ private class KStreamReduceProcessor extends ContextualProcessor> context) { + super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); store = context.getStateStore(storeName); @@ -81,9 +82,9 @@ public void process(final Record record) { LOG.warn( "Skipping record due to null key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", record.key(), record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index 683547237b4ee..a21f687c19f21 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -92,6 +92,7 @@ private class KStreamSessionWindowAggregateProcessor extends @Override public void init(final ProcessorContext, Change> context) { + super.init(context); final InternalProcessorContext, Change> internalProcessorContext = (InternalProcessorContext, Change>) context; metrics = (StreamsMetricsImpl) context.metrics(); @@ -147,7 +148,7 @@ public void process(final Record record) { } } - if (mergedWindow.end() < closeTime) { //TODO + if (mergedWindow.end() < closeTime) { LOG.warn( "Skipping record for expired window. " + "key=[{}] " + @@ -159,9 +160,9 @@ public void process(final Record record) { "expiration=[{}] " + "streamTime=[{}]", record.key(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L), timestamp, mergedWindow.start(), mergedWindow.end(), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index 3a51510d9cfd9..b6e3b24f4d73f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -24,9 +24,11 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.SlidingWindows; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; @@ -75,11 +77,10 @@ public void enableSendingOldValues() { sendOldValues = true; } - private class KStreamSlidingWindowAggregateProcessor implements Processor, Change> { + private class KStreamSlidingWindowAggregateProcessor extends ContextualProcessor, Change> { private TimestampedWindowStore windowStore; private TupleChangeForwarder, Agg> tupleForwarder; private StreamsMetricsImpl metrics; - private InternalProcessorContext internalProcessorContext; private Sensor lateRecordDropSensor; private Sensor droppedRecordsSensor; private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; @@ -87,7 +88,9 @@ private class KStreamSlidingWindowAggregateProcessor implements Processor, Change> context) { - internalProcessorContext = (InternalProcessorContext) context; + super.init(context); + final InternalProcessorContext, Change> internalProcessorContext = + (InternalProcessorContext, Change>) context; metrics = internalProcessorContext.metrics(); final String threadId = Thread.currentThread().getName(); lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor( @@ -109,11 +112,13 @@ public void init(final ProcessorContext, Change> context) { @Override public void process(final Record record) { if (record.key() == null || record.value() == null) { - //TODO -// log.warn( -// "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// value, context().topic(), context().partition(), context().offset() -// ); + log.warn( + "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -123,26 +128,25 @@ public void process(final Record record) { final long closeTime = observedStreamTime - windows.gracePeriodMs(); if (inputRecordTimestamp + 1L + windows.timeDifferenceMs() <= closeTime) { - //TODO -// log.warn( -// "Skipping record for expired window. " + -// "key=[{}] " + -// "topic=[{}] " + -// "partition=[{}] " + -// "offset=[{}] " + -// "timestamp=[{}] " + -// "window=[{},{}] " + -// "expiration=[{}] " + -// "streamTime=[{}]", -// key, -// context().topic(), -// context().partition(), -// context().offset(), -// context().timestamp(), -// inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp, -// closeTime, -// observedStreamTime -// ); + log.warn( + "Skipping record for expired window. " + + "key=[{}] " + + "topic=[{}] " + + "partition=[{}] " + + "offset=[{}] " + + "timestamp=[{}] " + + "window=[{},{}] " + + "expiration=[{}] " + + "streamTime=[{}]", + record.key(), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L), + record.timestamp(), + inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp, + closeTime, + observedStreamTime + ); lateRecordDropSensor.record(); return; } @@ -481,25 +485,25 @@ private void updateWindowAndForward(final Window window, sendOldValues ? oldAgg : null, newTimestamp); } else { -// log.warn( -// "Skipping record for expired window. " + -// "key=[{}] " + -// "topic=[{}] " + -// "partition=[{}] " + -// "offset=[{}] " + -// "timestamp=[{}] " + -// "window=[{},{}] " + -// "expiration=[{}] " + -// "streamTime=[{}]", -// key, -// context().topic(), -// context().partition(), -// context().offset(), -// context().timestamp(), -// windowStart, windowEnd, -// closeTime, -// observedStreamTime -// ); + log.warn( + "Skipping record for expired window. " + + "key=[{}] " + + "topic=[{}] " + + "partition=[{}] " + + "offset=[{}] " + + "timestamp=[{}] " + + "window=[{},{}] " + + "expiration=[{}] " + + "streamTime=[{}]", + record.key(), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L), + record.timestamp(), + windowStart, windowEnd, + closeTime, + observedStreamTime + ); lateRecordDropSensor.record(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 8cbfbc6141a25..6b58bb1f27823 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -42,7 +42,7 @@ import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; public class KStreamWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { - private final static Logger LOG = LoggerFactory.getLogger(KStreamWindowAggregate.class); + private final Logger log = LoggerFactory.getLogger(getClass()); private final String storeName; private final Windows windows; @@ -87,6 +87,7 @@ private class KStreamWindowAggregateProcessor extends @Override public void init(final ProcessorContext, Change> context) { + super.init(context); final InternalProcessorContext, Change> internalProcessorContext = (InternalProcessorContext, Change>) context; metrics = internalProcessorContext.metrics(); @@ -109,7 +110,7 @@ public void init(final ProcessorContext, Change> context) { @Override public void process(final Record record) { if (record.key() == null) { //TODO - LOG.warn( + log.warn( "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", record.value(), context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), @@ -155,7 +156,7 @@ public void process(final Record record) { sendOldValues ? oldAgg : null, newTimestamp); } else { - LOG.warn( + log.warn( "Skipping record for expired window. " + "key=[{}] " + "topic=[{}] " + diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index e0fce0bfd08bc..39d211d44198a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -94,9 +94,9 @@ public void process(final Record> record) { LOG.warn( "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index dafe4f46121c0..661508318bf69 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -89,9 +89,9 @@ public void process(final Record> record) { LOG.warn( "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 86c921e3bd7e9..65af581dd37fb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -88,9 +88,9 @@ public void process(final Record> record) { LOG.warn( "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index 93e7d60a32f72..3c9b5ac3d50d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -87,9 +87,9 @@ public void process(final Record> record) { LOG.warn( "Skipping record due to null key. change=[{}] topic=[{}] partition=[{}] offset=[{}]", record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index 29dca4012a776..4477c8627ed28 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -102,9 +102,9 @@ public void process(final Record record) { if (record.key() == null) { LOG.warn( "Skipping record due to null key. topic=[{}] partition=[{}] offset=[{}]", - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java index 4ef0b55680170..4dd44594ec02f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java @@ -18,7 +18,7 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.internals.CacheFlushListener; @@ -40,7 +40,7 @@ public void apply(final Windowed key, final ProcessorNode prev = context.currentNode(); context.setCurrentNode(myNode); try { - context.forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(key.window().end())); + context.forward(new Record<>(key, new Change<>(newValue, oldValue), key.window().end())); } finally { context.setCurrentNode(prev); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java index bc0328d8cc1ab..b5d5a6ee0d34d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java @@ -90,9 +90,9 @@ public void process(final Record> record) { LOG.warn( "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", record.value(), - context.recordMetadata().map(RecordMetadata::topic).orElse("<>"), - context.recordMetadata().map(RecordMetadata::partition).orElse(-1), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L) + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) ); droppedRecordsSensor.record(); return; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java index 0a13453b878f2..d987bc4e8de58 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.internals.Murmur3; @@ -105,20 +106,26 @@ public void process(final Record> record) { if (record.value().oldValue != null) { final KO oldForeignKey = foreignKeyExtractor.apply(record.value().oldValue); if (oldForeignKey == null) { -// LOG.warn( -// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// change.oldValue, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value().oldValue, + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } if (record.value().newValue != null) { final KO newForeignKey = foreignKeyExtractor.apply(record.value().newValue); if (newForeignKey == null) { -// LOG.warn( -// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// change.newValue, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value().newValue, + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -154,10 +161,13 @@ public void process(final Record> record) { } final KO newForeignKey = foreignKeyExtractor.apply(record.value().newValue); if (newForeignKey == null) { -// LOG.warn( -// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// change.newValue, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value().newValue, + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); } else { context().forward(record.withKey(newForeignKey).withValue(new SubscriptionWrapper<>(currentHash, instruction, record.key()))); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java index f7798ef0cf5d0..5a9cc6046362b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java @@ -40,25 +40,23 @@ */ public class SubscriptionJoinForeignProcessorSupplier implements - ProcessorSupplier, Change>>, CombinedKey, SubscriptionResponseWrapper> { + ProcessorSupplier, Change>>, K, SubscriptionResponseWrapper> { private final KTableValueGetterSupplier foreignValueGetterSupplier; - public SubscriptionJoinForeignProcessorSupplier( - final KTableValueGetterSupplier foreignValueGetterSupplier) { + public SubscriptionJoinForeignProcessorSupplier(final KTableValueGetterSupplier foreignValueGetterSupplier) { this.foreignValueGetterSupplier = foreignValueGetterSupplier; } @Override - public Processor, Change>>, CombinedKey, SubscriptionResponseWrapper> get() { + public Processor, Change>>, K, SubscriptionResponseWrapper> get() { - return new ContextualProcessor, Change>>, CombinedKey, SubscriptionResponseWrapper>() { + return new ContextualProcessor, Change>>, K, SubscriptionResponseWrapper>() { private KTableValueGetter foreignValues; @Override - public void init( - final ProcessorContext, SubscriptionResponseWrapper> context) { + public void init(final ProcessorContext> context) { super.init(context); foreignValues = foreignValueGetterSupplier.get(); foreignValues.init(context); @@ -96,8 +94,8 @@ public void process( case DELETE_KEY_AND_PROPAGATE: context().forward( record - .withValue( - new SubscriptionResponseWrapper(value.getHash(), null)) + .withKey(record.key().getPrimaryKey()) + .withValue(new SubscriptionResponseWrapper(value.getHash(), null)) .withTimestamp(resultTimestamp) ); break; @@ -109,8 +107,8 @@ public void process( context().forward( record - .withValue( - new SubscriptionResponseWrapper<>(value.getHash(), valueToSend)) + .withKey(record.key().getPrimaryKey()) + .withValue(new SubscriptionResponseWrapper<>(value.getHash(), valueToSend)) .withTimestamp(resultTimestamp) ); break; @@ -118,9 +116,8 @@ public void process( if (foreignValueAndTime != null) { context().forward( record - .withValue( - new SubscriptionResponseWrapper<>(value.getHash(), - foreignValueAndTime.value())) + .withKey(record.key().getPrimaryKey()) + .withValue(new SubscriptionResponseWrapper<>(value.getHash(), foreignValueAndTime.value())) .withTimestamp(resultTimestamp) ); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java index 7be67a0617075..b423becf4d6d9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.StoreBuilder; @@ -76,10 +77,13 @@ public void init(final ProcessorContext, Change> record) { if (record.key() == null) { -// LOG.warn( -// "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", -// value, context().topic(), context().partition(), context().offset() -// ); + LOG.warn( + "Skipping record due to null foreign key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", + record.value(), + context().recordMetadata().map(RecordMetadata::topic).orElse("<>"), + context().recordMetadata().map(RecordMetadata::partition).orElse(-1), + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L) + ); droppedRecordsSensor.record(); return; } @@ -106,10 +110,10 @@ public void process(final Record> record) { // note: key is non-nullable // note: newValue is non-nullable context().forward( - record.withKey( - new CombinedKey<>(record.key(), record.value().getPrimaryKey())) - .withValue(change) - .withTimestamp(newValue.timestamp()) + record + .withKey(new CombinedKey<>(record.key(), record.value().getPrimaryKey())) + .withValue(change) + .withTimestamp(newValue.timestamp()) ); } }; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java index 4c200a134bc24..806737bdadff7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java @@ -24,8 +24,8 @@ import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.kstream.internals.KStreamFlatTransform.KStreamFlatTransformProcessor; -import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.junit.Before; @@ -128,7 +128,7 @@ public void shouldGetFlatTransformProcessor() { EasyMock.expect(transformerSupplier.get()).andReturn(transformer); replayAll(); - final Processor processor = processorSupplier.get(); + final Processor processor = processorSupplier.getInternal(); verifyAll(); assertTrue(processor instanceof KStreamFlatTransformProcessor); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java index d0266309b2606..bbc4814294a42 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java @@ -24,8 +24,8 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.KStreamFlatTransformValues.KStreamFlatTransformValuesProcessor; +import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; @@ -127,7 +127,7 @@ public void shouldGetFlatTransformValuesProcessor() { EasyMock.expect(valueTransformerSupplier.get()).andReturn(valueTransformer); replayAll(); - final Processor processor = processorSupplier.get(); + final Processor processor = processorSupplier.getInternal(); verifyAll(); assertTrue(processor instanceof KStreamFlatTransformValuesProcessor); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index c3883dd129bc6..71f9d58e2f9d6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -47,12 +47,10 @@ import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; -import org.apache.kafka.streams.processor.internals.ToInternal; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; @@ -75,7 +73,6 @@ public class KStreamSessionWindowAggregateProcessorTest { private static final String STORE_NAME = "session-store"; private final String threadId = Thread.currentThread().getName(); - private final ToInternal toInternal = new ToInternal(); private final Initializer initializer = () -> 0L; private final Aggregator aggregator = (aggKey, value, aggregate) -> aggregate + 1; private final Merger sessionMerger = (aggKey, aggOne, aggTwo) -> aggOne + aggTwo; @@ -109,11 +106,12 @@ public void initializeStore() { new ThreadCache(new LogContext("testCache "), 100000, metrics), Time.SYSTEM ) { - @SuppressWarnings("unchecked") @Override - public void forward(final Object key, final Object value, final To to) { - toInternal.update(to); - results.add(new KeyValueTimestamp<>((Windowed) key, (Change) value, toInternal.timestamp())); + public , V1 extends Change> void forward( + final Record record + ) { + results.add( + new KeyValueTimestamp<>(record.key(), record.value(), record.timestamp())); } }; @@ -239,7 +237,6 @@ public void shouldRemoveMergedSessionsFromStateStore() { @Test public void shouldHandleMultipleSessionsAndMerging() { - context.setTime(0); processor.process(new Record<>("a", "1", 0)); processor.process(new Record<>("b", "1", 0)); processor.process(new Record<>("c", "1", 0)); @@ -614,11 +611,10 @@ private InternalMockProcessorContext, Change> createInter new ThreadCache(new LogContext("testCache "), 100000, streamsMetrics), Time.SYSTEM ) { - @SuppressWarnings("unchecked") @Override - public void forward(final Object key, final Object value, final To to) { - toInternal.update(to); - results.add(new KeyValueTimestamp<>((Windowed) key, (Change) value, toInternal.timestamp())); + public , V1 extends Change> void forward(final Record record) { + results.add( + new KeyValueTimestamp<>(record.key(), record.value(), record.timestamp())); } }; TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), streamsMetrics); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java index a77ff5269f0da..708d2b92d6078 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java @@ -276,7 +276,7 @@ private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtI join.init(context); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KTableKTableInnerJoin.class)) { - join.process(new Record<>(null, new Change<>("new", "old"), -3)); + join.process(new Record<>(null, new Change<>("new", "old"), 0)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 3a6343606a129..c77eecf1e636a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -433,7 +433,7 @@ private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtI join.init(context); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KTableKTableOuterJoin.class)) { - join.process(new Record<>(null, new Change<>("new", "old"), -3)); + join.process(new Record<>(null, new Change<>("new", "old"), 0)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java index 8de7e5d5d22d2..2fb2bb1e24ce1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java @@ -66,7 +66,7 @@ private void shouldLogAndMeterSkippedRecordsDueToNullLeftKey(final String builtI join.init(context); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KTableKTableRightJoin.class)) { - join.process(new Record<>(null, new Change<>("new", "old"), -3)); + join.process(new Record<>(null, new Change<>("new", "old"), 0L)); assertThat( appender.getMessages(), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java index 38cfacbc906e8..6fef1fe8b45ee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListenerTest.java @@ -16,29 +16,29 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.To; -import org.apache.kafka.streams.processor.internals.InternalProcessorContext; -import org.junit.Test; - import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.junit.Test; + public class SessionCacheFlushListenerTest { - @SuppressWarnings("unchecked") @Test public void shouldForwardKeyNewValueOldValueAndTimestamp() { - final InternalProcessorContext context = mock(InternalProcessorContext.class); + final InternalProcessorContext, Change> context = mock(InternalProcessorContext.class); expect(context.currentNode()).andReturn(null).anyTimes(); context.setCurrentNode(null); context.setCurrentNode(null); context.forward( - new Windowed<>("key", new SessionWindow(21L, 73L)), - new Change<>("newValue", "oldValue"), - To.all().withTimestamp(73L)); + new Record<>( + new Windowed<>("key", new SessionWindow(21L, 73L)), + new Change<>("newValue", "oldValue"), + 73L)); expectLastCall(); replay(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java index 3a2d1e7942c9b..3b4bab798ecfa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimestampedCacheFlushListenerTest.java @@ -16,17 +16,17 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.To; -import org.apache.kafka.streams.processor.internals.InternalProcessorContext; -import org.apache.kafka.streams.state.ValueAndTimestamp; -import org.junit.Test; - import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.junit.Test; + public class TimestampedCacheFlushListenerTest { @Test @@ -36,13 +36,16 @@ public void shouldForwardValueTimestampIfNewValueExists() { context.setCurrentNode(null); context.setCurrentNode(null); context.forward( - "key", - new Change<>("newValue", "oldValue"), - To.all().withTimestamp(42L)); + new Record<>( + "key", + new Change<>("newValue", "oldValue"), + 42L)); expectLastCall(); replay(context); - new TupleChangeCacheFlushListener<>(context).apply( + final TupleChangeCacheFlushListener flushListener = new TupleChangeCacheFlushListener<>( + context); + flushListener.apply( "key", ValueAndTimestamp.make("newValue", 42L), ValueAndTimestamp.make("oldValue", 21L), @@ -58,9 +61,10 @@ public void shouldForwardParameterTimestampIfNewValueIsNull() { context.setCurrentNode(null); context.setCurrentNode(null); context.forward( - "key", - new Change<>(null, "oldValue"), - To.all().withTimestamp(73L)); + new Record<>( + "key", + new Change<>(null, "oldValue"), + 73L)); expectLastCall(); replay(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java index 66126f9ab2657..77ecab273ee16 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java @@ -35,6 +35,7 @@ import java.util.Map; import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.collection.IsEmptyCollection.empty; @@ -147,8 +148,9 @@ public void shouldForwardWhenHashMatches() { processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue"), 0)); final List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); - assertThat(forwarded.get(0).record().key(), is("lhs1")); - assertThat(forwarded.get(0).record().value(), is("(lhsValue,rhsValue)")); + final Record record = forwarded.get(0).record(); + assertThat(record.key(), is("lhs1")); + assertThat(record.value(), is("(lhsValue,rhsValue)")); } @Test @@ -175,8 +177,9 @@ public void shouldEmitTombstoneForInnerJoinWhenRightIsNull() { processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, null), 0)); final List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); - assertThat(forwarded.get(0).record().key(), is("lhs1")); - assertThat(forwarded.get(0).record().value(), is(null)); + final Record record = forwarded.get(0).record(); + assertThat(record.key(), is("lhs1")); + assertThat(record.value(), nullValue()); } @Test @@ -230,7 +233,8 @@ public void shouldEmitTombstoneForLeftJoinWhenRightIsNullAndLeftIsNull() { processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, null), 0)); final List> forwarded = context.forwarded(); assertThat(forwarded.size(), is(1)); - assertThat(forwarded.get(0).record().key(), is("lhs1")); - assertThat(forwarded.get(0).record().value(), is(null)); + final Record record = forwarded.get(0).record(); + assertThat(record.key(), is("lhs1")); + assertThat(record.value(), nullValue()); } } diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java index d31dc5374d1a6..6cebc3c4a8ded 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java @@ -258,8 +258,8 @@ public void process(final Record record) { record.value(), record.headers(), record.timestamp(), - context.recordMetadata().map(RecordMetadata::offset).orElse(-1L), - context.recordMetadata().map(RecordMetadata::topic).orElse(null) + context().recordMetadata().map(RecordMetadata::offset).orElse(-1L), + context().recordMetadata().map(RecordMetadata::topic).orElse(null) )); context.forward(record); } From b557fe0ad24cd395f628eebbcaacf2984239fd9c Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 12 Apr 2021 17:24:41 +0100 Subject: [PATCH 29/40] fix duplicated store bug --- .../kstream/internals/graph/StatefulProcessorNode.java | 9 --------- .../integration/KStreamTransformIntegrationTest.java | 4 +++- 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java index 4df2fc9ba0ffd..ae7970654b576 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java @@ -98,14 +98,5 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { } } - // temporary hack until KIP-478 is fully implemented - final ProcessorSupplier oldProcessorSupplier = - processorParameters().processorSupplier(); - if (oldProcessorSupplier != null && oldProcessorSupplier.stores() != null) { - for (final StoreBuilder storeBuilder : oldProcessorSupplier.stores()) { - topologyBuilder.addStateStore(storeBuilder, processorName); - } - } - } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamTransformIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamTransformIntegrationTest.java index 7b13d9f3ecbda..170cd7ed5202f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamTransformIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamTransformIntegrationTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.TestInputTopic; +import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.KeyValue; @@ -77,7 +78,8 @@ private StoreBuilder> storeBuilder() { private void verifyResult(final List> expected) { final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.Integer()); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final Topology topology = builder.build(); + try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic inputTopic = driver.createInputTopic(topic, new IntegerSerializer(), new IntegerSerializer()); inputTopic.pipeKeyValueList(Arrays.asList( From dda022cb8a76d7c1e037a189952ac1b24a26c30e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 12 Apr 2021 17:37:10 +0100 Subject: [PATCH 30/40] fix missing context --- .../org/apache/kafka/streams/tests/SmokeTestUtil.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index 86989526f1fa1..78bcae67f3230 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; public class SmokeTestUtil { @@ -45,6 +46,7 @@ static ProcessorSupplier printProcessorSupplier( @Override public void init(final ProcessorContext context) { + super.init(context); System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); System.out.flush(); numRecordsProcessed = 0; @@ -60,11 +62,12 @@ public void process(final Record record) { System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); } - if (smallestOffset > context().recordMetadata().get().offset()) { - smallestOffset = context().recordMetadata().get().offset(); + final long offset = context().recordMetadata().map(RecordMetadata::offset).orElse(-1L); + if (smallestOffset > offset) { + smallestOffset = offset; } - if (largestOffset < context().recordMetadata().get().offset()) { - largestOffset = context().recordMetadata().get().offset(); + if (largestOffset < offset) { + largestOffset = offset; } } From 1aaadf2a9cb2fcb89d4f540f76f74cb389ad4a56 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 12 Apr 2021 19:26:30 +0100 Subject: [PATCH 31/40] add mock for old processor api --- .../apache/kafka/streams/kstream/KStream.java | 32 ++-- .../kstream/internals/KStreamImpl.java | 27 ---- .../kafka/streams/StreamsBuilderTest.java | 53 ++++--- .../GlobalKTableIntegrationTest.java | 33 ++-- .../GlobalThreadShutDownOrderTest.java | 11 +- .../integration/RestoreIntegrationTest.java | 60 ++++--- .../StoreUpgradeIntegrationTest.java | 77 ++++----- ...caughtExceptionHandlerIntegrationTest.java | 9 +- .../kstream/RepartitionTopicNamingTest.java | 18 ++- .../kstream/internals/AbstractStreamTest.java | 25 ++- .../internals/GlobalKTableJoinsTest.java | 8 +- .../internals/KGroupedStreamImplTest.java | 34 ++-- .../internals/KGroupedTableImplTest.java | 12 +- .../kstream/internals/KStreamBranchTest.java | 21 ++- .../kstream/internals/KStreamFilterTest.java | 17 +- .../kstream/internals/KStreamFlatMapTest.java | 25 ++- .../internals/KStreamFlatMapValuesTest.java | 24 ++- .../KStreamGlobalKTableJoinTest.java | 8 +- .../KStreamGlobalKTableLeftJoinTest.java | 8 +- .../kstream/internals/KStreamImplTest.java | 80 +++++----- .../internals/KStreamKStreamJoinTest.java | 65 ++++---- .../internals/KStreamKStreamLeftJoinTest.java | 16 +- .../internals/KStreamKTableJoinTest.java | 8 +- .../internals/KStreamKTableLeftJoinTest.java | 8 +- .../kstream/internals/KStreamMapTest.java | 21 ++- .../internals/KStreamMapValuesTest.java | 13 +- .../internals/KStreamSelectKeyTest.java | 21 ++- .../KStreamSlidingWindowAggregateTest.java | 73 +++++---- .../internals/KStreamTransformTest.java | 17 +- .../internals/KStreamTransformValuesTest.java | 4 +- .../internals/KStreamWindowAggregateTest.java | 10 +- .../internals/KTableAggregateTest.java | 12 +- .../kstream/internals/KTableFilterTest.java | 27 ++-- .../kstream/internals/KTableImplTest.java | 37 +++-- .../kstream/internals/KTableMapKeysTest.java | 4 +- .../internals/KTableMapValuesTest.java | 29 ++-- .../kstream/internals/KTableSourceTest.java | 4 +- .../internals/KTableTransformValuesTest.java | 6 +- .../SessionWindowedKStreamImplTest.java | 29 ++-- .../SlidingWindowedKStreamImplTest.java | 29 ++-- .../TimeWindowedKStreamImplTest.java | 29 ++-- .../internals/RepartitionOptimizingTest.java | 9 +- .../processor/internals/StreamThreadTest.java | 125 +++++++-------- .../kafka/streams/tests/SmokeTestUtil.java | 20 ++- .../kafka/test/MockOldApiProcessor.java | 147 ++++++++++++++++++ .../apache/kafka/test/MockOldProcessor.java | 65 ++++++++ .../kafka/test/MockOldProcessorSupplier.java | 73 +++++++++ .../org/apache/kafka/test/MockProcessor.java | 8 - .../kafka/test/MockProcessorSupplier.java | 4 - .../kafka/streams/TopologyTestDriverTest.java | 4 +- 50 files changed, 859 insertions(+), 640 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/test/MockOldApiProcessor.java create mode 100644 streams/src/test/java/org/apache/kafka/test/MockOldProcessor.java create mode 100644 streams/src/test/java/org/apache/kafka/test/MockOldProcessorSupplier.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index e58114a83b37c..afbaf03d23da0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -46,7 +46,7 @@ * A {@code KStream} can be transformed record by record, joined with another {@code KStream}, {@link KTable}, * {@link GlobalKTable}, or can be aggregated into a {@link KTable}. * Kafka Streams DSL can be mixed-and-matched with Processor API (PAPI) (c.f. {@link Topology}) via - * {@link #process(ProcessorSupplier, String...) process(...)}, + * {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) process(...)}, * {@link #transform(TransformerSupplier, String...) transform(...)}, and * {@link #transformValues(ValueTransformerSupplier, String...) transformValues(...)}. * @@ -694,28 +694,28 @@ KStream flatMapValues(final ValueMapperWithKey action); /** * Perform an action on each record of {@code KStream}. - * This is a stateless record-by-record operation (cf. {@link #process(ProcessorSupplier, String...)}). + * This is a stateless record-by-record operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...)}). * Note that this is a terminal operation that returns void. * * @param action an action to perform on each record * @param named a {@link Named} config used to name the processor in the topology - * @see #process(ProcessorSupplier, String...) + * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ void foreach(final ForeachAction action, final Named named); /** * Perform an action on each record of {@code KStream}. - * This is a stateless record-by-record operation (cf. {@link #process(ProcessorSupplier, String...)}). + * This is a stateless record-by-record operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...)}). *

* Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection) * and returns an unchanged stream. @@ -723,14 +723,14 @@ KStream flatMapValues(final ValueMapperWithKey peek(final ForeachAction action); /** * Perform an action on each record of {@code KStream}. - * This is a stateless record-by-record operation (cf. {@link #process(ProcessorSupplier, String...)}). + * This is a stateless record-by-record operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...)}). *

* Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection) * and returns an unchanged stream. @@ -739,7 +739,7 @@ KStream flatMapValues(final ValueMapperWithKey peek(final ForeachAction action, final Named named); @@ -3199,7 +3199,7 @@ KStream leftJoin(final GlobalKTable globalTable, * @see #flatTransform(TransformerSupplier, String...) * @see #transformValues(ValueTransformerSupplier, String...) * @see #transformValues(ValueTransformerWithKeySupplier, String...) - * @see #process(ProcessorSupplier, String...) + * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ KStream transform(final TransformerSupplier> transformerSupplier, final String... stateStoreNames); @@ -3327,7 +3327,7 @@ KStream transform(final TransformerSupplier KStream transform(final TransformerSupplier> transformerSupplier, final Named named, @@ -3454,7 +3454,7 @@ KStream transform(final TransformerSupplier KStream flatTransform(final TransformerSupplier>> transformerSupplier, final String... stateStoreNames); @@ -3581,7 +3581,7 @@ KStream flatTransform(final TransformerSupplier KStream flatTransform(final TransformerSupplier>> transformerSupplier, final Named named, @@ -4713,10 +4713,4 @@ void process(final org.apache.kafka.streams.processor.ProcessorSupplier void process(final ProcessorSupplier processorSupplier, - final String... stateStoreNames); - - void process(final ProcessorSupplier processorSupplier, - final Named named, - final String... stateStoreNames); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 950d2671863db..16c8f05e54b98 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1493,31 +1493,4 @@ public void process(final org.apache.kafka.streams.processor.ProcessorSupplier void process(final ProcessorSupplier processorSupplier, - final String... stateStoreNames) { - process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); - } - - @Override - public void process(final ProcessorSupplier processorSupplier, - final Named named, - final String... stateStoreNames) { - Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); - Objects.requireNonNull(named, "named can't be null"); - Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); - ApiUtils.checkSupplier(processorSupplier); - for (final String stateStoreName : stateStoreNames) { - Objects.requireNonNull(stateStoreName, "stateStoreNames can't be null"); - } - - final String name = new NamedInternal(named).name(); - final StatefulProcessorNode processNode = new StatefulProcessorNode<>( - name, - new ProcessorParameters<>(processorSupplier, name), - stateStoreNames); - - builder.addGraphNode(graphNode, processNode); - } } diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java index a8874359d7007..757be649decf0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -16,6 +16,24 @@ */ package org.apache.kafka.streams; +import static java.util.Arrays.asList; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -46,33 +64,14 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.test.MockMapper; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockPredicate; -import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.NoopValueTransformer; import org.apache.kafka.test.NoopValueTransformerWithKey; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.regex.Pattern; - -import static java.util.Arrays.asList; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - public class StreamsBuilderTest { private static final String STREAM_TOPIC = "stream-topic"; @@ -296,7 +295,7 @@ public void shouldProcessingFromSinkTopic() { final KStream source = builder.stream("topic-source"); source.to("topic-sink"); - final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier processorSupplier = new MockOldProcessorSupplier<>(); source.process(processorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -316,10 +315,10 @@ public void shouldProcessViaThroughTopic() { final KStream source = builder.stream("topic-source"); final KStream through = source.through("topic-sink"); - final MockProcessorSupplier sourceProcessorSupplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier sourceProcessorSupplier = new MockOldProcessorSupplier<>(); source.process(sourceProcessorSupplier); - final MockProcessorSupplier throughProcessorSupplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier throughProcessorSupplier = new MockOldProcessorSupplier<>(); through.process(throughProcessorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -337,10 +336,10 @@ public void shouldProcessViaRepartitionTopic() { final KStream source = builder.stream("topic-source"); final KStream through = source.repartition(); - final MockProcessorSupplier sourceProcessorSupplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier sourceProcessorSupplier = new MockOldProcessorSupplier<>(); source.process(sourceProcessorSupplier); - final MockProcessorSupplier throughProcessorSupplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier throughProcessorSupplier = new MockOldProcessorSupplier<>(); through.process(throughProcessorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -362,7 +361,7 @@ public void shouldMergeStreams() { final KStream source2 = builder.stream(topic2); final KStream merged = source1.merge(source2); - final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier processorSupplier = new MockOldProcessorSupplier<>(); merged.process(processorSupplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -855,7 +854,7 @@ public void shouldUseSpecifiedNameForMergeOperation() { @Test public void shouldUseSpecifiedNameForProcessOperation() { builder.stream(STREAM_TOPIC) - .process(new MockProcessorSupplier<>(), Named.as("test-processor")); + .process(new MockOldProcessorSupplier<>(), Named.as("test-processor")); builder.build(); final ProcessorTopology topology = builder.internalTopologyBuilder.rewriteTopology(new StreamsConfig(props)).buildTopology(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java index 36d67c14f5fbd..bf87955957c74 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java @@ -16,6 +16,19 @@ */ package org.apache.kafka.streams.integration; +import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertNotNull; + +import java.io.IOException; +import java.time.Duration; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; import kafka.utils.MockTime; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.LongSerializer; @@ -41,7 +54,7 @@ import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.AfterClass; @@ -52,20 +65,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; -import java.time.Duration; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import static java.util.Collections.singletonList; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.IsEqual.equalTo; -import static org.junit.Assert.assertNotNull; - @Category({IntegrationTest.class}) public class GlobalKTableIntegrationTest { private static final int NUM_BROKERS = 1; @@ -94,7 +93,7 @@ public static void closeCluster() { private String streamTopic; private GlobalKTable globalTable; private KStream stream; - private MockProcessorSupplier supplier; + private MockOldProcessorSupplier supplier; @Rule public TestName testName = new TestName(); @@ -117,7 +116,7 @@ public void before() throws Exception { .withValueSerde(Serdes.String())); final Consumed stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long()); stream = builder.stream(streamTopic, stringLongConsumed); - supplier = new MockProcessorSupplier<>(); + supplier = new MockOldProcessorSupplier<>(); } @After diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java index 4c717e395ac76..5b44eeda6e5bc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java @@ -30,9 +30,8 @@ import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; @@ -197,7 +196,7 @@ private void populateTopics(final String topicName) { } - private class GlobalStoreProcessor implements Processor { + private class GlobalStoreProcessor implements Processor { private KeyValueStore store; private final String storeName; @@ -207,12 +206,12 @@ private class GlobalStoreProcessor implements Processor context) { + public void init(final ProcessorContext context) { store = context.getStateStore(storeName); } @Override - public void process(final Record record) { + public void process(final String key, final Long value) { firstRecordProcessed = true; } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 299a75c08443c..6d1539b0b14ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -16,6 +16,28 @@ */ package org.apache.kafka.streams.integration; +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForCompletion; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForStandbyCompletion; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -40,11 +62,10 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -63,31 +84,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import java.io.File; -import java.io.IOException; -import java.time.Duration; -import java.util.Collections; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import org.junit.rules.TestName; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForCompletion; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForStandbyCompletion; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.IsEqual.equalTo; -import static org.junit.Assert.assertTrue; - @Category({IntegrationTest.class}) public class RestoreIntegrationTest { private static final int NUM_BROKERS = 1; @@ -431,7 +429,7 @@ static int numStoresClosed() { } } - public static class KeyValueStoreProcessor implements Processor { + public static class KeyValueStoreProcessor implements Processor { private final String topic; private final CountDownLatch processorLatch; @@ -444,14 +442,14 @@ public static class KeyValueStoreProcessor implements Processor context) { + public void init(final ProcessorContext context) { this.store = context.getStateStore(topic); } @Override - public void process(final Record record) { - if (record.key() != null) { - store.put(record.key(), record.value()); + public void process(final Integer key, final Integer value) { + if (key != null) { + store.put(key, value); processorLatch.countDown(); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java index 20dc383942757..591ded64c251d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java @@ -16,20 +16,29 @@ */ package org.apache.kafka.streams.integration; +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; + +import java.io.IOException; +import java.time.Duration; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; -import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; @@ -51,16 +60,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; -import java.time.Duration; -import java.util.LinkedList; -import java.util.List; -import java.util.Properties; - -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; - @Category({IntegrationTest.class}) public class StoreUpgradeIntegrationTest { private static final String STORE_NAME = "store"; @@ -954,112 +953,114 @@ private void processKeyValueAndVerifyWindowedCountWithTimestamp(final K k "Could not get expected result in time."); } - private static class KeyValueProcessor implements Processor { + private static class KeyValueProcessor implements Processor { private KeyValueStore store; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { store = context.getStateStore(STORE_NAME); } @Override - public void process(final Record record) { + public void process(final Integer key, final Integer value) { final long newCount; - final Long oldCount = store.get(record.key()); + final Long oldCount = store.get(key); if (oldCount != null) { newCount = oldCount + 1L; } else { newCount = 1L; } - store.put(record.key(), newCount); + store.put(key, newCount); } @Override public void close() {} } - private static class TimestampedKeyValueProcessor implements Processor { + private static class TimestampedKeyValueProcessor extends AbstractProcessor { private TimestampedKeyValueStore store; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { + super.init(context); store = context.getStateStore(STORE_NAME); } @Override - public void process(final Record record) { + public void process(final Integer key, final Integer value) { final long newCount; - final ValueAndTimestamp oldCountWithTimestamp = store.get(record.key()); + final ValueAndTimestamp oldCountWithTimestamp = store.get(key); final long newTimestamp; if (oldCountWithTimestamp == null) { newCount = 1L; - newTimestamp = record.timestamp(); + newTimestamp = context.timestamp(); } else { newCount = oldCountWithTimestamp.value() + 1L; - newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), record.timestamp()); + newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), context.timestamp()); } - store.put(record.key(), ValueAndTimestamp.make(newCount, newTimestamp)); + store.put(key, ValueAndTimestamp.make(newCount, newTimestamp)); } @Override public void close() {} } - private static class WindowedProcessor implements Processor { + private static class WindowedProcessor implements Processor { private WindowStore store; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { store = context.getStateStore(STORE_NAME); } @Override - public void process(final Record record) { + public void process(final Integer key, final Integer value) { final long newCount; - final Long oldCount = store.fetch(record.key(), record.key() < 10 ? 0L : 100000L); + final Long oldCount = store.fetch(key, key < 10 ? 0L : 100000L); if (oldCount != null) { newCount = oldCount + 1L; } else { newCount = 1L; } - store.put(record.key(), newCount, record.key() < 10 ? 0L : 100000L); + store.put(key, newCount, key < 10 ? 0L : 100000L); } @Override public void close() {} } - private static class TimestampedWindowedProcessor implements Processor { + private static class TimestampedWindowedProcessor extends AbstractProcessor { private TimestampedWindowStore store; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { + super.init(context); store = context.getStateStore(STORE_NAME); } @Override - public void process(final Record record) { + public void process(final Integer key, final Integer value) { final long newCount; - final ValueAndTimestamp oldCountWithTimestamp = store.fetch(record.key(), record.key() < 10 ? 0L : 100000L); + final ValueAndTimestamp oldCountWithTimestamp = store.fetch(key, key < 10 ? 0L : 100000L); final long newTimestamp; if (oldCountWithTimestamp == null) { newCount = 1L; - newTimestamp = record.timestamp(); + newTimestamp = context().timestamp(); } else { newCount = oldCountWithTimestamp.value() + 1L; - newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), record.timestamp()); + newTimestamp = Math.max(oldCountWithTimestamp.timestamp(), context().timestamp()); } - store.put(record.key(), ValueAndTimestamp.make(newCount, newTimestamp), record.key() < 10 ? 0L : 100000L); + store.put(key, ValueAndTimestamp.make(newCount, newTimestamp), key < 10 ? 0L : 100000L); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index ae020950d9e40..2e89c02647f25 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -52,8 +52,7 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Named; -import org.apache.kafka.streams.processor.api.ContextualProcessor; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.test.IntegrationTest; @@ -223,7 +222,7 @@ private void produceMessages(final long timestamp, final String streamOneInput, timestamp); } - private static class ShutdownProcessor extends ContextualProcessor { + private static class ShutdownProcessor extends AbstractProcessor { final List valueList; ShutdownProcessor(final List valueList) { @@ -231,8 +230,8 @@ private static class ShutdownProcessor extends ContextualProcessor record) { - valueList.add(record + " " + context.taskId()); + public void process(final String key, final String value) { + valueList.add(value + " " + context.taskId()); if (throwError.get()) { throw new StreamsException(Thread.currentThread().getName()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java index 3c251d1aaefd7..cf4c0350806ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java @@ -23,8 +23,8 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.errors.TopologyException; -import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.junit.Test; import java.time.Duration; @@ -506,7 +506,7 @@ private Topology buildTopology(final String optimizationConfig) { } - private static class SimpleProcessor implements Processor { + private static class SimpleProcessor implements Processor { final List valueList; @@ -515,8 +515,16 @@ private static class SimpleProcessor implements Processor record) { - valueList.add(record.value()); + public void init(final ProcessorContext context) { + } + + @Override + public void process(final String key, final String value) { + valueList.add(value); + } + + @Override + public void close() { } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java index fedf98e998ca6..58a2ec2df30d5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java @@ -16,12 +16,16 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.junit.Assert.assertTrue; + +import java.util.Random; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.processor.api.ContextualProcessor; -import org.apache.kafka.streams.processor.api.Record; -import org.apache.kafka.test.NoopValueTransformerWithKey; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; @@ -30,19 +34,14 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode; import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.test.MockOldProcessorSupplier; +import org.apache.kafka.test.NoopValueTransformerWithKey; import org.junit.Test; -import java.util.Random; - -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; -import static org.junit.Assert.assertTrue; - public class AbstractStreamTest { //TODO @@ -75,7 +74,7 @@ public void testToInternalValueTransformerWithKeySupplierSuppliesNewTransformers public void testShouldBeExtensible() { final StreamsBuilder builder = new StreamsBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); final String topicName = "topic"; final ExtendedKStream stream = new ExtendedKStream<>(builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()))); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java index b088d7ecc80ec..557798f4312bf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java @@ -26,7 +26,7 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; @@ -58,7 +58,7 @@ public void setUp() { @Test public void shouldLeftJoinWithStream() { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream .leftJoin(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER) .process(supplier); @@ -73,7 +73,7 @@ public void shouldLeftJoinWithStream() { @Test public void shouldInnerJoinWithStream() { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream .join(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER) .process(supplier); @@ -86,7 +86,7 @@ public void shouldInnerJoinWithStream() { } private void verifyJoin(final Map> expected, - final MockProcessorSupplier supplier) { + final MockOldProcessorSupplier supplier) { final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 05f5b6d75d726..bee5d084eab7e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -44,7 +44,7 @@ import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; @@ -202,7 +202,7 @@ public void shouldNotHaveInvalidStoreNameOnSlidingWindowedAggregate() { @Test public void shouldCountSlidingWindows() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); groupedStream .windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(500L), ofMillis(2000L))) .count(Materialized.as("aggregate-by-key-windowed")) @@ -214,7 +214,7 @@ public void shouldCountSlidingWindows() { @Test public void shouldCountSlidingWindowsWithInternalStoreName() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); groupedStream .windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(500L), ofMillis(2000L))) .count() @@ -224,7 +224,7 @@ public void shouldCountSlidingWindowsWithInternalStoreName() { doCountSlidingWindows(supplier); } - private void doCountSlidingWindows(final MockProcessorSupplier, Long, ?, ?> supplier) { + private void doCountSlidingWindows(final MockOldProcessorSupplier, Long> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -319,7 +319,7 @@ private void doCountSlidingWindows(final MockProcessorSupplier ))); } - private void doAggregateSessionWindows(final MockProcessorSupplier, Integer, Windowed, Integer> supplier) { + private void doAggregateSessionWindows(final MockOldProcessorSupplier, Integer> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -345,7 +345,7 @@ private void doAggregateSessionWindows(final MockProcessorSupplier, Integer, Windowed, Integer> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Integer> supplier = new MockOldProcessorSupplier<>(); final KTable, Integer> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .aggregate( @@ -363,7 +363,7 @@ public void shouldAggregateSessionWindows() { @Test public void shouldAggregateSessionWindowsWithInternalStoreName() { - final MockProcessorSupplier, Integer, Windowed, Integer> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Integer> supplier = new MockOldProcessorSupplier<>(); final KTable, Integer> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .aggregate( @@ -376,7 +376,7 @@ public void shouldAggregateSessionWindowsWithInternalStoreName() { doAggregateSessionWindows(supplier); } - private void doCountSessionWindows(final MockProcessorSupplier, Long, Windowed, Long> supplier) { + private void doCountSessionWindows(final MockOldProcessorSupplier, Long> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = @@ -403,7 +403,7 @@ private void doCountSessionWindows(final MockProcessorSupplier, @Test public void shouldCountSessionWindows() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); final KTable, Long> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .count(Materialized.as("session-store")); @@ -414,7 +414,7 @@ public void shouldCountSessionWindows() { @Test public void shouldCountSessionWindowsWithInternalStoreName() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); final KTable, Long> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .count(); @@ -423,7 +423,7 @@ public void shouldCountSessionWindowsWithInternalStoreName() { assertNull(table.queryableStoreName()); } - private void doReduceSessionWindows(final MockProcessorSupplier, String, Windowed, String> supplier) { + private void doReduceSessionWindows(final MockOldProcessorSupplier, String> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -449,7 +449,7 @@ private void doReduceSessionWindows(final MockProcessorSupplier @Test public void shouldReduceSessionWindows() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); final KTable, String> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .reduce((value1, value2) -> value1 + ":" + value2, Materialized.as("session-store")); @@ -460,7 +460,7 @@ public void shouldReduceSessionWindows() { @Test public void shouldReduceSessionWindowsWithInternalStoreName() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); final KTable, String> table = groupedStream .windowedBy(SessionWindows.with(ofMillis(30))) .reduce((value1, value2) -> value1 + ":" + value2); @@ -724,7 +724,7 @@ public void shouldAggregateAndMaterializeResults() { @Test public void shouldAggregateWithDefaultSerdes() { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); groupedStream .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER) .toStream() @@ -757,7 +757,7 @@ private void processData(final TopologyTestDriver driver) { inputTopic.pipeInput("3", (String) null); } - private void doCountWindowed(final MockProcessorSupplier, Long, ?, ?> supplier) { + private void doCountWindowed(final MockOldProcessorSupplier, Long> supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, new StringSerializer(), new StringSerializer()); @@ -792,7 +792,7 @@ private void doCountWindowed(final MockProcessorSupplier, Long @Test public void shouldCountWindowed() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); groupedStream .windowedBy(TimeWindows.of(ofMillis(500L))) .count(Materialized.as("aggregate-by-key-windowed")) @@ -804,7 +804,7 @@ public void shouldCountWindowed() { @Test public void shouldCountWindowedWithInternalStoreName() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); groupedStream .windowedBy(TimeWindows.of(ofMillis(500L))) .count() diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java index da56674b8bd57..b4da9eabb020b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -36,7 +36,7 @@ import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockMapper; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; @@ -126,8 +126,8 @@ public void shouldNotAllowInvalidStoreNameOnReduce() { Materialized.as(INVALID_STORE_NAME))); } - private MockProcessorSupplier getReducedResults(final KTable inputKTable) { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private MockOldProcessorSupplier getReducedResults(final KTable inputKTable) { + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); inputKTable .toStream() .process(supplier); @@ -172,7 +172,7 @@ public void shouldReduce() { MockReducer.INTEGER_SUBTRACTOR, Materialized.as("reduced")); - final MockProcessorSupplier supplier = getReducedResults(reduced); + final MockOldProcessorSupplier supplier = getReducedResults(reduced); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { assertReduced(supplier.theCapturedProcessor().lastValueAndTimestampPerKey(), topic, driver); assertEquals(reduced.queryableStoreName(), "reduced"); @@ -194,7 +194,7 @@ public void shouldReduceWithInternalStoreName() { .groupBy(intProjection) .reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR); - final MockProcessorSupplier supplier = getReducedResults(reduced); + final MockOldProcessorSupplier supplier = getReducedResults(reduced); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { assertReduced(supplier.theCapturedProcessor().lastValueAndTimestampPerKey(), topic, driver); assertNull(reduced.queryableStoreName()); @@ -219,7 +219,7 @@ public void shouldReduceAndMaterializeResults() { .withKeySerde(Serdes.String()) .withValueSerde(Serdes.Integer())); - final MockProcessorSupplier supplier = getReducedResults(reduced); + final MockOldProcessorSupplier supplier = getReducedResults(reduced); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { assertReduced(supplier.theCapturedProcessor().lastValueAndTimestampPerKey(), topic, driver); { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index 3531db78c43d2..b7f36981208ed 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -16,25 +16,24 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertEquals; + +import java.util.List; +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.util.List; -import java.util.Properties; - -import static org.junit.Assert.assertEquals; - public class KStreamBranchTest { private final String topicName = "topic"; @@ -59,7 +58,7 @@ public void testKStreamBranch() { assertEquals(3, branches.length); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); for (final KStream branch : branches) { branch.process(supplier); } @@ -71,7 +70,7 @@ public void testKStreamBranch() { } } - final List> processors = supplier.capturedProcessors(3); + final List> processors = supplier.capturedProcessors(3); assertEquals(3, processors.get(0).processed().size()); assertEquals(1, processors.get(1).processed().size()); assertEquals(2, processors.get(2).processed().size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index ed7297df2b865..28184664feade 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -16,23 +16,22 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertEquals; + +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.util.Properties; - -import static org.junit.Assert.assertEquals; - public class KStreamFilterTest { private final String topicName = "topic"; @@ -46,7 +45,7 @@ public void testFilter() { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; final KStream stream; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.filter(isMultipleOfThree).process(supplier); @@ -67,7 +66,7 @@ public void testFilterNot() { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; final KStream stream; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.filterNot(isMultipleOfThree).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index 431de64bae1f2..7e3122e77a3ee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -16,29 +16,27 @@ */ package org.apache.kafka.streams.kstream.internals; -import java.util.List; +import static org.junit.Assert.assertEquals; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.KeyValueTimestamp; -import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Properties; - -import static org.junit.Assert.assertEquals; - public class KStreamFlatMapTest { private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); @@ -59,9 +57,8 @@ public void testFlatMap() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream; - final MockProcessorSupplier> supplier; + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); - supplier = new MockProcessorSupplier<>(); stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.flatMap(mapper).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index b27d293cb3247..c0375ec1df0ab 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -16,28 +16,26 @@ */ package org.apache.kafka.streams.kstream.internals; -import java.util.List; +import static org.junit.Assert.assertArrayEquals; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValueTimestamp; -import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Properties; - -import static org.junit.Assert.assertArrayEquals; - public class KStreamFlatMapValuesTest { private final String topicName = "topic"; private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); @@ -57,7 +55,7 @@ public void testFlatMapValues() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer())); - final MockProcessorSupplier> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream.flatMapValues(mapper).process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -93,7 +91,7 @@ public void testFlatMapValuesWithKeys() { final int[] expectedKeys = {0, 1, 2, 3}; final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer())); - final MockProcessorSupplier> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream.flatMapValues(mapper).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java index c2212555b4141..fa36c0f14d12c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java @@ -28,8 +28,8 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.After; @@ -52,7 +52,7 @@ public class KStreamGlobalKTableJoinTest { private final int[] expectedKeys = {0, 1, 2, 3}; private TopologyTestDriver driver; - private MockProcessor processor; + private MockOldProcessor processor; private StreamsBuilder builder; @Before @@ -63,7 +63,7 @@ public void setUp() { final GlobalKTable table; // value of stream optionally contains key of table final KeyValueMapper keyMapper; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); final Consumed streamConsumed = Consumed.with(Serdes.Integer(), Serdes.String()); final Consumed tableConsumed = Consumed.with(Serdes.String(), Serdes.String()); stream = builder.stream(streamTopic, streamConsumed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java index f0cc2656f9a9c..e79d68fe56f37 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java @@ -28,8 +28,8 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.After; @@ -51,7 +51,7 @@ public class KStreamGlobalKTableLeftJoinTest { private final String globalTableTopic = "globalTableTopic"; private final int[] expectedKeys = {0, 1, 2, 3}; - private MockProcessor processor; + private MockOldProcessor processor; private TopologyTestDriver driver; private StreamsBuilder builder; @@ -63,7 +63,7 @@ public void setUp() { final GlobalKTable table; // value of stream optionally contains key of table final KeyValueMapper keyMapper; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); final Consumed streamConsumed = Consumed.with(Serdes.Integer(), Serdes.String()); final Consumed tableConsumed = Consumed.with(Serdes.String(), Serdes.String()); stream = builder.stream(streamTopic, streamConsumed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 605d935233b05..397901c3a3f70 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -16,6 +16,35 @@ */ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static java.util.Arrays.asList; +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.hamcrest.core.IsNull.notNullValue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; @@ -55,7 +84,6 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.SourceNode; @@ -63,47 +91,17 @@ import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.test.TestRecord; import org.apache.kafka.test.MockMapper; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import static java.time.Duration.ofMillis; -import static java.util.Arrays.asList; -import static java.util.Collections.emptyMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.IsInstanceOf.instanceOf; -import static org.hamcrest.core.IsNull.notNullValue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - public class KStreamImplTest { private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String()); - private final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + private final MockOldProcessorSupplier processorSupplier = new MockOldProcessorSupplier<>(); private final TransformerSupplier> transformerSupplier = () -> new Transformer>() { @Override @@ -1289,9 +1287,9 @@ public void testNumProcesses() { stream4.to("topic-5"); - streams2[1].through("topic-6").process(new MockProcessorSupplier<>()); + streams2[1].through("topic-6").process(new MockOldProcessorSupplier<>()); - streams2[1].repartition().process(new MockProcessorSupplier<>()); + streams2[1].repartition().process(new MockOldProcessorSupplier<>()); assertEquals(2 + // sources 2 + // stream1 @@ -1504,7 +1502,7 @@ public void shouldSendDataToDynamicTopics() { inputTopic.pipeInput("a", "v2"); inputTopic.pipeInput("b", "v1"); } - final List> mockProcessors = processorSupplier.capturedProcessors(2); + final List> mockProcessors = processorSupplier.capturedProcessors(2); assertThat(mockProcessors.get(0).processed(), equalTo(asList(new KeyValueTimestamp<>("a", "v1", 0), new KeyValueTimestamp<>("a", "v2", 0)))); assertThat(mockProcessors.get(1).processed(), equalTo(Collections.singletonList(new KeyValueTimestamp<>("b", "v1", 0)))); @@ -2382,7 +2380,7 @@ public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueWithKeySupp public void shouldNotAllowNullProcessSupplierOnProcess() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null)); + () -> testStream.process(null)); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2390,7 +2388,7 @@ public void shouldNotAllowNullProcessSupplierOnProcess() { public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null, "storeName")); + () -> testStream.process(null, "storeName")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2398,7 +2396,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithStores() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null, Named.as("processor"))); + () -> testStream.process(null, Named.as("processor"))); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } @@ -2406,7 +2404,7 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() { public void shouldNotAllowNullProcessSupplierOnProcessWithNamedAndStores() { final NullPointerException exception = assertThrows( NullPointerException.class, - () -> testStream.process((ProcessorSupplier) null, Named.as("processor"), "stateStore")); + () -> testStream.process(null, Named.as("processor"), "stateStore")); assertThat(exception.getMessage(), equalTo("processorSupplier can't be null")); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index 0d78a5f9d628b..b1bc0c6d730d4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -16,6 +16,23 @@ */ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Properties; +import java.util.Set; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -23,6 +40,7 @@ import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.TopologyWrapper; @@ -34,33 +52,14 @@ import org.apache.kafka.streams.processor.internals.InternalTopicConfig; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; -import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Properties; -import java.util.Set; - -import static java.time.Duration.ofMillis; -import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - public class KStreamKStreamJoinTest { private final static KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0]; @@ -346,7 +345,7 @@ private void runJoin(final StreamJoined streamJoined, final KStream left = builder.stream("left", Consumed.with(Serdes.String(), Serdes.Integer())); final KStream right = builder.stream("right", Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); final KStream joinedStream; joinedStream = left.join( @@ -362,7 +361,7 @@ private void runJoin(final StreamJoined streamJoined, driver.createInputTopic("left", new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopicRight = driver.createInputTopic("right", new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); inputTopicLeft.pipeInput("A", 1, 1L); inputTopicLeft.pipeInput("B", 1, 2L); @@ -384,7 +383,7 @@ public void testJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); joined = stream1.join( @@ -405,7 +404,7 @@ public void testJoin() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty // w1 = {} @@ -495,7 +494,7 @@ public void testOuterJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -516,7 +515,7 @@ public void testOuterJoin() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty; this should produce two items // w1 = {} @@ -609,7 +608,7 @@ public void testWindowing() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -631,7 +630,7 @@ public void testWindowing() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); long time = 0L; // push two items to the primary stream; the other window is empty; this should produce no items @@ -1137,7 +1136,7 @@ public void testAsymmetricWindowingAfter() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -1161,7 +1160,7 @@ public void testAsymmetricWindowingAfter() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); long time = 1000L; // push four items with increasing timestamps to the primary stream; the other window is empty; this should produce no items @@ -1388,7 +1387,7 @@ public void testAsymmetricWindowingBefore() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -1411,7 +1410,7 @@ public void testAsymmetricWindowingBefore() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); long time = 1000L; // push four items with increasing timestamps to the primary stream; the other window is empty; this should produce no items diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 504f3adf43f0b..c54036d86fa2c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -38,8 +38,8 @@ import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.StreamJoined; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; @@ -61,7 +61,7 @@ public void testLeftJoin() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -83,7 +83,7 @@ public void testLeftJoin() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty // w1 {} @@ -154,7 +154,7 @@ public void testWindowing() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -176,7 +176,7 @@ public void testWindowing() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); final long time = 0L; // push two items to the primary stream; the other window is empty; this should produce two left-join items @@ -206,7 +206,7 @@ public void testWindowing() { private void testUpperWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, - final MockProcessor processor) { + final MockOldProcessor processor) { long time; final TestInputTopic inputTopic1 = @@ -330,7 +330,7 @@ private void testUpperWindowBound(final int[] expectedKeys, private void testLowerWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, - final MockProcessor processor) { + final MockOldProcessor processor) { long time; final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer()); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java index c97c0be21f47c..a0212717f4ddd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java @@ -45,8 +45,8 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.After; @@ -62,11 +62,11 @@ public class KStreamKTableJoinTest { private TestInputTopic inputTableTopic; private final int[] expectedKeys = {0, 1, 2, 3}; - private MockProcessor processor; + private MockOldProcessor processor; private TopologyTestDriver driver; private StreamsBuilder builder; private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); @Before public void setUp() { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index 996e250412373..891b3c38b5071 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -27,8 +27,8 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.After; @@ -56,7 +56,7 @@ public class KStreamKTableLeftJoinTest { private final int[] expectedKeys = {0, 1, 2, 3}; private TopologyTestDriver driver; - private MockProcessor processor; + private MockOldProcessor processor; private StreamsBuilder builder; @Before @@ -66,7 +66,7 @@ public void setUp() { final KStream stream; final KTable table; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); stream = builder.stream(streamTopic, consumed); table = builder.table(tableTopic, consumed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 5510af17a7ead..18452fad0ebc5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -16,26 +16,25 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertEquals; + +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.KeyValueTimestamp; -import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; - -import static org.junit.Assert.assertEquals; - public class KStreamMapTest { private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); @@ -45,7 +44,7 @@ public void testMap() { final String topicName = "topic"; final int[] expectedKeys = new int[] {0, 1, 2, 3}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); final KStream stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())); stream.map((key, value) -> KeyValue.pair(value, key)).process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index c83ba5d403793..eddc4a9d4b345 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -16,27 +16,26 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertArrayEquals; + +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.util.Properties; - -import static org.junit.Assert.assertArrayEquals; - public class KStreamMapValuesTest { private final String topicName = "topic"; - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java index 9d639a0b00d91..97a24ff90a0ed 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java @@ -16,27 +16,26 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertEquals; + +import java.time.Duration; +import java.time.Instant; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import static org.junit.Assert.assertEquals; - public class KStreamSelectKeyTest { private final String topicName = "topic_key_select"; private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.Integer()); @@ -57,7 +56,7 @@ public void testSelectKey() { final KStream stream = builder.stream(topicName, Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream.selectKey((key, value) -> keyMap.get(value)).process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java index 68740cb83c956..c0cde93e6b95b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java @@ -16,6 +16,30 @@ */ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static java.util.Arrays.asList; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; @@ -24,6 +48,7 @@ import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TestOutputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; @@ -40,15 +65,14 @@ import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; -import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.InMemoryWindowBytesStoreSupplier; import org.apache.kafka.streams.state.internals.InMemoryWindowStore; import org.apache.kafka.streams.test.TestRecord; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.StreamsTestUtils; import org.hamcrest.Matcher; @@ -56,31 +80,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; - -import static java.time.Duration.ofMillis; -import static java.util.Arrays.asList; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.CoreMatchers.hasItems; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - @RunWith(Parameterized.class) public class KStreamSlidingWindowAggregateTest { @@ -117,7 +116,7 @@ public void testAggregateSmallInput() { MockAggregator.TOSTRING_ADDER, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = @@ -173,7 +172,7 @@ public void testReduceSmallInput() { MockReducer.STRING_ADDER, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = @@ -231,7 +230,7 @@ public void testAggregateLargeInput() { Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -403,7 +402,7 @@ public void testJoin() { Materialized.as(storeSupplier2) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table1.toStream().process(supplier); table2.toStream().process(supplier); @@ -419,7 +418,7 @@ public void testJoin() { inputTopic1.pipeInput("B", "2", 11L); inputTopic1.pipeInput("C", "3", 12L); - final List, String, Windowed, String>> processors = supplier.capturedProcessors(3); + final List, String>> processors = supplier.capturedProcessors(3); processors.get(0).checkAndClearProcessResult( // left windows created by the first set of records to table 1 @@ -504,7 +503,7 @@ public void testEarlyRecordsSmallInput() { MockAggregator.TOSTRING_ADDER, Materialized.>as("topic-Canonized").withValueSerde(Serdes.String()) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -555,7 +554,7 @@ public void testEarlyRecordsRepeatedInput() { MockAggregator.TOSTRING_ADDER, Materialized.>as("topic-Canonized").withValueSerde(Serdes.String()) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -607,7 +606,7 @@ public void testEarlyRecordsLargeInput() { Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -812,7 +811,7 @@ public void testAggregateRandomInput() { }, Materialized.as(storeSupplier) ); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table.toStream().process(supplier); final long seed = new Random().nextLong(); final Random shuffle = new Random(seed); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index 52087145e7dbd..c7efd1d1ecdaa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -16,6 +16,11 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.junit.Assert.assertEquals; + +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; @@ -30,16 +35,10 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.To; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; - -import static org.junit.Assert.assertEquals; - public class KStreamTransformTest { private static final String TOPIC_NAME = "topic"; private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.Integer()); @@ -73,7 +72,7 @@ public void close() { } final int[] expectedKeys = {1, 10, 100, 1000}; - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier processor = new MockOldProcessorSupplier<>(); final KStream stream = builder.stream(TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer())); stream.transform(transformerSupplier).process(processor); @@ -134,7 +133,7 @@ public void close() { } final int[] expectedKeys = {1, 10, 100, 1000}; - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier processor = new MockOldProcessorSupplier<>(); final KStream stream = builder.stream(TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer())); stream.transform(transformerSupplier).process(processor); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index eaf718f7b9644..03712fefbd882 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -32,7 +32,7 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.easymock.EasyMockRunner; import org.easymock.Mock; @@ -43,7 +43,7 @@ @RunWith(EasyMockRunner.class) public class KStreamTransformValuesTest { private final String topicName = "topic"; - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.Integer()); @Mock(MockType.NICE) private ProcessorContext context; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index c1f9d656968ef..7c60febbf1985 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -40,8 +40,8 @@ import org.apache.kafka.streams.test.TestRecord; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.hamcrest.Matcher; import org.junit.Test; @@ -79,7 +79,7 @@ public void testAggBasic() { .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { @@ -157,7 +157,7 @@ public void testJoin() { .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); table1.toStream().process(supplier); final KTable, String> table2 = builder @@ -180,7 +180,7 @@ public void testJoin() { inputTopic1.pipeInput("D", "4", 3L); inputTopic1.pipeInput("A", "1", 9L); - final List, String, ?, ?>> processors = supplier.capturedProcessors(3); + final List, String>> processors = supplier.capturedProcessors(3); processors.get(0).checkAndClearProcessResult( new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(0, 10)), "0+1", 0), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index a1d322c144b69..452c4e52d3073 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -34,8 +34,8 @@ import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockMapper; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.TestUtils; import org.junit.Test; import java.util.Properties; @@ -53,7 +53,7 @@ public class KTableAggregateTest { private final Serde stringSerde = Serdes.String(); private final Consumed consumed = Consumed.with(stringSerde, stringSerde); private final Grouped stringSerialized = Grouped.with(stringSerde, stringSerde); - private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + private final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); private final static Properties CONFIG = mkProperties(mkMap( mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory("kafka-test").getAbsolutePath()))); @@ -168,7 +168,7 @@ public void testAggRepartition() { private static void testCountHelper(final StreamsBuilder builder, final String input, - final MockProcessorSupplier supplier) { + final MockOldProcessorSupplier supplier) { try ( final TopologyTestDriver driver = new TopologyTestDriver( builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { @@ -228,7 +228,7 @@ public void testCountWithInternalStore() { public void testRemoveOldBeforeAddNew() { final StreamsBuilder builder = new StreamsBuilder(); final String input = "count-test-input"; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); builder .table(input, consumed) @@ -252,7 +252,7 @@ public void testRemoveOldBeforeAddNew() { final TestInputTopic inputTopic = driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor proc = supplier.theCapturedProcessor(); + final MockOldProcessor proc = supplier.theCapturedProcessor(); inputTopic.pipeInput("11", "A", 10L); inputTopic.pipeInput("12", "B", 8L); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 321c6a00897de..5446a566b54b0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -16,6 +16,15 @@ */ package org.apache.kafka.streams.kstream.internals; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import java.util.Properties; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -36,23 +45,13 @@ import org.apache.kafka.test.MockApiProcessor; import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockMapper; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.List; -import java.util.Properties; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - @SuppressWarnings("unchecked") public class KTableFilterTest { private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.Integer()); @@ -70,7 +69,7 @@ private void doTestKTable(final StreamsBuilder builder, final KTable table2, final KTable table3, final String topic) { - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); table3.toStream().process(supplier); @@ -85,7 +84,7 @@ private void doTestKTable(final StreamsBuilder builder, inputTopic.pipeInput("B", null, 15L); } - final List> processors = supplier.capturedProcessors(2); + final List> processors = supplier.capturedProcessors(2); processors.get(0).checkAndClearProcessResult(new KeyValueTimestamp<>("A", null, 10), new KeyValueTimestamp<>("B", 2, 5), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 6649d73784d95..7e2a4e87e68de 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -16,6 +16,18 @@ */ package org.apache.kafka.streams.kstream.internals; +import static java.util.Arrays.asList; +import static org.easymock.EasyMock.mock; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; + +import java.lang.reflect.Field; +import java.util.List; +import java.util.Properties; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -46,27 +58,14 @@ import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockMapper; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; import org.junit.Test; -import java.lang.reflect.Field; -import java.util.List; -import java.util.Properties; - -import static java.util.Arrays.asList; -import static org.easymock.EasyMock.mock; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThrows; - @SuppressWarnings("unchecked") public class KTableImplTest { private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String()); @@ -91,7 +90,7 @@ public void testKTable() { final KTable table1 = builder.table(topic1, consumed); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); table1.toStream().process(supplier); final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); @@ -115,7 +114,7 @@ public void testKTable() { inputTopic.pipeInput("A", "06", 8L); } - final List> processors = supplier.capturedProcessors(4); + final List> processors = supplier.capturedProcessors(4); assertEquals(asList( new KeyValueTimestamp<>("A", "01", 5), new KeyValueTimestamp<>("B", "02", 100), @@ -159,7 +158,7 @@ public void testMaterializedKTable() { final KTable table1 = builder.table(topic1, consumed, Materialized.as("fred")); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); table1.toStream().process(supplier); final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); @@ -183,7 +182,7 @@ public void testMaterializedKTable() { inputTopic.pipeInput("A", "06", 8L); } - final List> processors = supplier.capturedProcessors(4); + final List> processors = supplier.capturedProcessors(4); assertEquals(asList( new KeyValueTimestamp<>("A", "01", 5), new KeyValueTimestamp<>("B", "02", 100), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java index 0542af6977b77..a911ad3636ba7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java @@ -26,7 +26,7 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; @@ -59,7 +59,7 @@ public void testMapKeysConvertingToStream() { final int[] originalKeys = new int[] {1, 2, 3}; final String[] values = new String[] {"V_ONE", "V_TWO", "V_THREE"}; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); convertedStream.process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index b726ccab7dd6d..60c7f437ea040 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -16,6 +16,16 @@ */ package org.apache.kafka.streams.kstream.internals; +import static java.util.Arrays.asList; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; + +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; @@ -34,21 +44,10 @@ import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.MockApiProcessor; import org.apache.kafka.test.MockApiProcessorSupplier; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; - -import static java.util.Arrays.asList; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; - @SuppressWarnings("unchecked") public class KTableMapValuesTest { private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.String()); @@ -56,7 +55,7 @@ public class KTableMapValuesTest { private void doTestKTable(final StreamsBuilder builder, final String topic1, - final MockProcessorSupplier supplier) { + final MockOldProcessorSupplier supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); @@ -79,7 +78,7 @@ public void testKTable() { final KTable table1 = builder.table(topic1, consumed); final KTable table2 = table1.mapValues(value -> value.charAt(0) - 48); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); doTestKTable(builder, topic1, supplier); @@ -97,7 +96,7 @@ public void testQueryableKTable() { Materialized.>as("anyName") .withValueSerde(Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); table2.toStream().process(supplier); doTestKTable(builder, topic1, supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index a409e19d13af0..b5ad162867206 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -39,7 +39,7 @@ import org.apache.kafka.streams.test.TestRecord; import org.apache.kafka.test.MockApiProcessor; import org.apache.kafka.test.MockApiProcessorSupplier; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Ignore; import org.junit.Test; @@ -68,7 +68,7 @@ public void testKTable() { final KTable table1 = builder.table(topic1, Consumed.with(Serdes.String(), Serdes.Integer())); - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); table1.toStream().process(supplier); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java index e39ca7bdc2f92..a55f395d99425 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java @@ -39,7 +39,7 @@ import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.NoOpValueTransformerWithKeySupplier; import org.apache.kafka.test.TestUtils; @@ -77,7 +77,7 @@ public class KTableTransformValuesTest { private static final Consumed CONSUMED = Consumed.with(Serdes.String(), Serdes.String()); private TopologyTestDriver driver; - private MockProcessorSupplier capture; + private MockOldProcessorSupplier capture; private StreamsBuilder builder; @Mock(MockType.NICE) private KTableImpl parent; @@ -104,7 +104,7 @@ public void cleanup() { @Before public void setUp() { - capture = new MockProcessorSupplier<>(); + capture = new MockOldProcessorSupplier<>(); builder = new StreamsBuilder(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java index c4b3d60581148..fe8ee2c2f4770 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java @@ -17,12 +17,22 @@ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertThrows; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Grouped; @@ -35,25 +45,14 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.ValueAndTimestamp; -import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -import static java.time.Duration.ofMillis; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertThrows; - public class SessionWindowedKStreamImplTest { private static final String TOPIC = "input"; private final StreamsBuilder builder = new StreamsBuilder(); @@ -80,7 +79,7 @@ public void shouldCountSessionWindowedWithCachingEnabled() { } private void shouldCountSessionWindowed() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); stream.count() .toStream() .process(supplier); @@ -106,7 +105,7 @@ private void shouldCountSessionWindowed() { @Test public void shouldReduceWindowed() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); stream.reduce(MockReducer.STRING_ADDER) .toStream() .process(supplier); @@ -132,7 +131,7 @@ public void shouldReduceWindowed() { @Test public void shouldAggregateSessionWindowed() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); stream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, sessionMerger, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java index 4497b8c902661..a53782931506b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java @@ -17,11 +17,21 @@ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static java.time.Instant.ofEpochMilli; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertThrows; + +import java.util.Arrays; +import java.util.List; +import java.util.Properties; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Grouped; @@ -35,25 +45,14 @@ import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; -import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Properties; - -import static java.time.Duration.ofMillis; -import static java.time.Instant.ofEpochMilli; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertThrows; - public class SlidingWindowedKStreamImplTest { private static final String TOPIC = "input"; @@ -71,7 +70,7 @@ public void before() { @Test public void shouldCountSlidingWindows() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); windowedStream .count() .toStream() @@ -112,7 +111,7 @@ public void shouldCountSlidingWindows() { @Test public void shouldReduceSlidingWindows() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); windowedStream .reduce(MockReducer.STRING_ADDER) .toStream() @@ -153,7 +152,7 @@ public void shouldReduceSlidingWindows() { @Test public void shouldAggregateSlidingWindows() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); windowedStream .aggregate( MockInitializer.STRING_INIT, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java index dcd31d288ae4f..afda9000da795 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java @@ -17,11 +17,21 @@ package org.apache.kafka.streams.kstream.internals; +import static java.time.Duration.ofMillis; +import static java.time.Instant.ofEpochMilli; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertThrows; + +import java.util.Arrays; +import java.util.List; +import java.util.Properties; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Grouped; @@ -33,25 +43,14 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; -import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Before; import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Properties; - -import static java.time.Duration.ofMillis; -import static java.time.Instant.ofEpochMilli; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertThrows; - public class TimeWindowedKStreamImplTest { private static final String TOPIC = "input"; private final StreamsBuilder builder = new StreamsBuilder(); @@ -68,7 +67,7 @@ public void before() { @Test public void shouldCountWindowed() { - final MockProcessorSupplier, Long, Windowed, Long> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, Long> supplier = new MockOldProcessorSupplier<>(); windowedStream .count() .toStream() @@ -93,7 +92,7 @@ public void shouldCountWindowed() { @Test public void shouldReduceWindowed() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); windowedStream .reduce(MockReducer.STRING_ADDER) .toStream() @@ -118,7 +117,7 @@ public void shouldReduceWindowed() { @Test public void shouldAggregateWindowed() { - final MockProcessorSupplier, String, Windowed, String> supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier, String> supplier = new MockOldProcessorSupplier<>(); windowedStream .aggregate( MockInitializer.STRING_INIT, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java index 767a075330c90..d82f6c9d1a855 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java @@ -44,8 +44,7 @@ import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.StreamJoined; -import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.test.StreamsTestUtils; import org.junit.After; @@ -258,7 +257,7 @@ private List> getKeyValues() { return keyValueList; } - private static class SimpleProcessor implements Processor { + private static class SimpleProcessor extends AbstractProcessor { final List valueList; @@ -267,8 +266,8 @@ private static class SimpleProcessor implements Processor record) { - valueList.add(record.value()); + public void process(final String key, final String value) { + valueList.add(value); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 2123efea6d4de..a156640beb99a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -16,6 +16,57 @@ */ package org.apache.kafka.streams.processor.internals; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkProperties; +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId; +import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.mock; +import static org.easymock.EasyMock.niceMock; +import static org.easymock.EasyMock.verify; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.startsWith; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.File; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; @@ -60,15 +111,12 @@ import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.LogAndSkipOnInvalidTimestamp; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TaskMetadata; import org.apache.kafka.streams.processor.ThreadMetadata; -import org.apache.kafka.streams.processor.api.ContextualProcessor; -import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.assignment.ReferenceContainer; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; @@ -89,58 +137,6 @@ import org.junit.Test; import org.slf4j.Logger; -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; -import static java.util.Collections.singleton; -import static java.util.Collections.singletonMap; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.common.utils.Utils.mkProperties; -import static org.apache.kafka.common.utils.Utils.mkSet; -import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId; -import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.easymock.EasyMock.mock; -import static org.easymock.EasyMock.niceMock; -import static org.easymock.EasyMock.verify; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.CoreMatchers.startsWith; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - public class StreamThreadTest { private final static String APPLICATION_ID = "stream-thread-test"; @@ -1812,14 +1808,14 @@ public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled() { public void shouldPunctuateActiveTask() { final List punctuatedStreamTime = new ArrayList<>(); final List punctuatedWallClockTime = new ArrayList<>(); - final ProcessorSupplier punctuateProcessor = - () -> new ContextualProcessor() { + final org.apache.kafka.streams.processor.ProcessorSupplier punctuateProcessor = + () -> new org.apache.kafka.streams.processor.AbstractProcessor() { @Override - public void process(final Record record) { + public void process(final Object key, final Object value) { } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { context.schedule(Duration.ofMillis(100L), PunctuationType.STREAM_TIME, punctuatedStreamTime::add); context.schedule(Duration.ofMillis(100L), PunctuationType.WALL_CLOCK_TIME, punctuatedWallClockTime::add); } @@ -1898,8 +1894,13 @@ public void close() {} }; final List peekedContextTime = new ArrayList<>(); - final ProcessorSupplier peekProcessor = - () -> (Processor) record -> peekedContextTime.add(record.timestamp()); + final org.apache.kafka.streams.processor.ProcessorSupplier peekProcessor = + () -> new org.apache.kafka.streams.processor.AbstractProcessor() { + @Override + public void process(final Object key, final Object value) { + peekedContextTime.add(context.timestamp()); + } + }; internalStreamsBuilder.stream(Collections.singleton(topic1), consumed) .transform(punctuateProcessor) diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index 78bcae67f3230..4304e34901bf7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -24,28 +24,26 @@ import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.api.ContextualProcessor; -import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.api.Record; -import org.apache.kafka.streams.processor.api.RecordMetadata; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; public class SmokeTestUtil { final static int END = Integer.MAX_VALUE; - static ProcessorSupplier printProcessorSupplier(final String topic) { + static ProcessorSupplier printProcessorSupplier(final String topic) { return printProcessorSupplier(topic, ""); } - static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { - return () -> new ContextualProcessor() { + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return () -> new AbstractProcessor() { private int numRecordsProcessed = 0; private long smallestOffset = Long.MAX_VALUE; private long largestOffset = Long.MIN_VALUE; @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); System.out.flush(); @@ -55,14 +53,14 @@ public void init(final ProcessorContext context) { } @Override - public void process(final Record record) { + public void process(final Object key, final Object value) { numRecordsProcessed++; if (numRecordsProcessed % 100 == 0) { System.out.printf("%s: %s%n", name, Instant.now()); System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); } - final long offset = context().recordMetadata().map(RecordMetadata::offset).orElse(-1L); + final long offset = context().offset(); if (smallestOffset > offset) { smallestOffset = offset; } diff --git a/streams/src/test/java/org/apache/kafka/test/MockOldApiProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockOldApiProcessor.java new file mode 100644 index 0000000000000..680433e3e8af9 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockOldApiProcessor.java @@ -0,0 +1,147 @@ +/* + * 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.test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.streams.KeyValueTimestamp; +import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class MockOldApiProcessor implements Processor { + + private final ArrayList> processed = new ArrayList<>(); + private final Map> lastValueAndTimestampPerKey = new HashMap<>(); + + private final ArrayList punctuatedStreamTime = new ArrayList<>(); + private final ArrayList punctuatedSystemTime = new ArrayList<>(); + + private Cancellable scheduleCancellable; + + private final PunctuationType punctuationType; + private final long scheduleInterval; + + private boolean commitRequested = false; + private ProcessorContext context; + + public MockOldApiProcessor(final PunctuationType punctuationType, + final long scheduleInterval) { + this.punctuationType = punctuationType; + this.scheduleInterval = scheduleInterval; + } + + public MockOldApiProcessor() { + this(PunctuationType.STREAM_TIME, -1); + } + + public void checkAndClearProcessResult(final KeyValueTimestamp... expected) { + assertThat("the number of outputs:" + processed, processed.size(), is(expected.length)); + for (int i = 0; i < expected.length; i++) { + assertThat("output[" + i + "]:", processed.get(i), is(expected[i])); + } + + processed.clear(); + } + + public void requestCommit() { + commitRequested = true; + } + + public void checkEmptyAndClearProcessResult() { + assertThat("the number of outputs:", processed.size(), is(0)); + processed.clear(); + } + + public void checkAndClearPunctuateResult(final PunctuationType type, final long... expected) { + final ArrayList punctuated = type == PunctuationType.STREAM_TIME ? punctuatedStreamTime : punctuatedSystemTime; + assertThat("the number of outputs:", punctuated.size(), is(expected.length)); + + for (int i = 0; i < expected.length; i++) { + assertThat("output[" + i + "]:", punctuated.get(i), is(expected[i])); + } + + processed.clear(); + } + + public ArrayList> processed() { + return processed; + } + + public Map> lastValueAndTimestampPerKey() { + return lastValueAndTimestampPerKey; + } + + public List punctuatedStreamTime() { + return punctuatedStreamTime; + } + + public Cancellable scheduleCancellable() { + return scheduleCancellable; + } + + public ProcessorContext context() { + return context; + } + + public void context(final ProcessorContext context) { + this.context = context; + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + if (scheduleInterval > 0L) { + scheduleCancellable = context.schedule( + Duration.ofMillis(scheduleInterval), + punctuationType, + (punctuationType == PunctuationType.STREAM_TIME ? punctuatedStreamTime : punctuatedSystemTime)::add + ); + } + } + + @Override + public void process(KIn key, VIn value) { + final KeyValueTimestamp keyValueTimestamp = new KeyValueTimestamp<>(key, value, context.timestamp()); + + if (value != null) { + lastValueAndTimestampPerKey.put(key, ValueAndTimestamp.make(value, context.timestamp())); + } else { + lastValueAndTimestampPerKey.remove(key); + } + + processed.add(keyValueTimestamp); + + if (commitRequested) { + context.commit(); + commitRequested = false; + } + } + + @Override + public void close() { + + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockOldProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockOldProcessor.java new file mode 100644 index 0000000000000..4b5bc3a6ead16 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockOldProcessor.java @@ -0,0 +1,65 @@ +/* + * 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.test; + +import java.util.ArrayList; +import java.util.Map; +import org.apache.kafka.streams.KeyValueTimestamp; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class MockOldProcessor implements Processor { + private final MockOldApiProcessor delegate; + + public MockOldProcessor(final PunctuationType punctuationType, + final long scheduleInterval) { + delegate = new MockOldApiProcessor<>(punctuationType, scheduleInterval); + } + + public MockOldProcessor() { + delegate = new MockOldApiProcessor<>(); + } + + @Override + public void init(final ProcessorContext context) { + delegate.init(context); + } + + @Override + public void process(KIn key, VIn value) { + delegate.process(key, value); + } + + @Override + public void close() { + delegate.close(); + } + + public void checkAndClearProcessResult(final KeyValueTimestamp... expected) { + delegate.checkAndClearProcessResult(expected); + } + + public Map> lastValueAndTimestampPerKey() { + return delegate.lastValueAndTimestampPerKey(); + } + + public ArrayList> processed() { + return delegate.processed(); + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockOldProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockOldProcessorSupplier.java new file mode 100644 index 0000000000000..2e0545d83b2b4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockOldProcessorSupplier.java @@ -0,0 +1,73 @@ +/* + * 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.test; + +import static org.junit.Assert.assertEquals; + +import java.util.ArrayList; +import java.util.List; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.PunctuationType; + +public class MockOldProcessorSupplier implements ProcessorSupplier { + + private final long scheduleInterval; + private final PunctuationType punctuationType; + private final List> processors = new ArrayList<>(); + + public MockOldProcessorSupplier() { + this(-1L); + } + + public MockOldProcessorSupplier(final long scheduleInterval) { + this(scheduleInterval, PunctuationType.STREAM_TIME); + } + + public MockOldProcessorSupplier(final long scheduleInterval, final PunctuationType punctuationType) { + this.scheduleInterval = scheduleInterval; + this.punctuationType = punctuationType; + } + + @Override + public Processor get() { + final MockOldProcessor processor = new MockOldProcessor<>(punctuationType, scheduleInterval); + + // to keep tests simple, ignore calls from ApiUtils.checkSupplier + if (!StreamsTestUtils.isCheckSupplierCall()) { + processors.add(processor); + } + + return processor; + } + + // get the captured processor assuming that only one processor gets returned from this supplier + public MockOldProcessor theCapturedProcessor() { + return capturedProcessors(1).get(0); + } + + public int capturedProcessorsCount() { + return processors.size(); + } + + // get the captured processors with the expected number + public List> capturedProcessors(final int expectedNumberOfProcessors) { + assertEquals(expectedNumberOfProcessors, processors.size()); + + return processors; + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java index 45a1b974495fa..fbf9c65945315 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -54,14 +54,6 @@ public void checkAndClearProcessResult(final KeyValueTimestamp... expected delegate.checkAndClearProcessResult(expected); } - public void requestCommit() { - delegate.requestCommit(); - } - - public void checkEmptyAndClearProcessResult() { - delegate.checkEmptyAndClearProcessResult(); - } - public void checkAndClearPunctuateResult(final PunctuationType type, final long... expected) { delegate.checkAndClearPunctuateResult(type, expected); } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java index b1f9a2899bf52..e174a5f13f7aa 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java @@ -62,10 +62,6 @@ public MockProcessor theCapturedProcessor() { return capturedProcessors(1).get(0); } - public int capturedProcessorsCount() { - return processors.size(); - } - // get the captured processors with the expected number public List> capturedProcessors(final int expectedNumberOfProcessors) { assertEquals(expectedNumberOfProcessors, processors.size()); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java index 6cebc3c4a8ded..d31dc5374d1a6 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java @@ -258,8 +258,8 @@ public void process(final Record record) { record.value(), record.headers(), record.timestamp(), - context().recordMetadata().map(RecordMetadata::offset).orElse(-1L), - context().recordMetadata().map(RecordMetadata::topic).orElse(null) + context.recordMetadata().map(RecordMetadata::offset).orElse(-1L), + context.recordMetadata().map(RecordMetadata::topic).orElse(null) )); context.forward(record); } From 2c9285ec746afed8191a7d217897cb56c7a3da90 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 15 Apr 2021 09:47:04 +0100 Subject: [PATCH 32/40] align types ordering --- .../CogroupedStreamAggregateBuilder.java | 80 +++++++++---------- .../GroupedStreamAggregateBuilder.java | 2 +- .../kstream/internals/KGroupedStreamImpl.java | 2 +- .../kstream/internals/KStreamAggregate.java | 38 ++++----- .../KStreamAggregateProcessorSupplier.java | 4 +- .../kstream/internals/KStreamReduce.java | 2 +- .../KStreamSessionWindowAggregate.java | 2 +- .../KStreamSlidingWindowAggregate.java | 2 +- .../internals/KStreamWindowAggregate.java | 2 +- .../streams/kstream/internals/KTableImpl.java | 2 +- .../kstream/internals/KTablePassThrough.java | 2 +- 11 files changed, 69 insertions(+), 69 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java index 29f631d4fbc79..d0ce08b12c2ab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java @@ -48,13 +48,13 @@ class CogroupedStreamAggregateBuilder { CogroupedStreamAggregateBuilder(final InternalStreamsBuilder builder) { this.builder = builder; } - KTable build(final Map, Aggregator> groupPatterns, - final Initializer initializer, - final NamedInternal named, - final StoreBuilder storeBuilder, - final Serde keySerde, - final Serde valueSerde, - final String queryableName) { + KTable build(final Map, Aggregator> groupPatterns, + final Initializer initializer, + final NamedInternal named, + final StoreBuilder storeBuilder, + final Serde keySerde, + final Serde valueSerde, + final String queryableName) { processRepartitions(groupPatterns, storeBuilder); final Collection processors = new ArrayList<>(); final Collection parentProcessors = new ArrayList<>(); @@ -79,14 +79,14 @@ KTable build(final Map, Aggregator KTable build(final Map, Aggregator> groupPatterns, - final Initializer initializer, - final NamedInternal named, - final StoreBuilder storeBuilder, - final Serde keySerde, - final Serde valueSerde, - final String queryableName, - final Windows windows) { + KTable build(final Map, Aggregator> groupPatterns, + final Initializer initializer, + final NamedInternal named, + final StoreBuilder storeBuilder, + final Serde keySerde, + final Serde valueSerde, + final String queryableName, + final Windows windows) { processRepartitions(groupPatterns, storeBuilder); final Collection processors = new ArrayList<>(); @@ -116,15 +116,15 @@ KTable build(final Map return createTable(processors, parentProcessors, named, keySerde, valueSerde, queryableName, storeBuilder.name()); } - KTable build(final Map, Aggregator> groupPatterns, - final Initializer initializer, - final NamedInternal named, - final StoreBuilder storeBuilder, - final Serde keySerde, - final Serde valueSerde, - final String queryableName, - final SessionWindows sessionWindows, - final Merger sessionMerger) { + KTable build(final Map, Aggregator> groupPatterns, + final Initializer initializer, + final NamedInternal named, + final StoreBuilder storeBuilder, + final Serde keySerde, + final Serde valueSerde, + final String queryableName, + final SessionWindows sessionWindows, + final Merger sessionMerger) { processRepartitions(groupPatterns, storeBuilder); final Collection processors = new ArrayList<>(); final Collection parentProcessors = new ArrayList<>(); @@ -155,14 +155,14 @@ KTable build(final Map, Aggregator KTable build(final Map, Aggregator> groupPatterns, - final Initializer initializer, - final NamedInternal named, - final StoreBuilder storeBuilder, - final Serde keySerde, - final Serde valueSerde, - final String queryableName, - final SlidingWindows slidingWindows) { + KTable build(final Map, Aggregator> groupPatterns, + final Initializer initializer, + final NamedInternal named, + final StoreBuilder storeBuilder, + final Serde keySerde, + final Serde valueSerde, + final String queryableName, + final SlidingWindows slidingWindows) { processRepartitions(groupPatterns, storeBuilder); final Collection parentProcessors = new ArrayList<>(); final Collection processors = new ArrayList<>(); @@ -222,13 +222,13 @@ private void processRepartitions(final Map, Aggregator< } @SuppressWarnings("unchecked") - KTable createTable(final Collection processors, - final Collection parentProcessors, - final NamedInternal named, - final Serde keySerde, - final Serde valueSerde, - final String queryableName, - final String storeName) { + KTable createTable(final Collection processors, + final Collection parentProcessors, + final NamedInternal named, + final Serde keySerde, + final Serde valueSerde, + final String queryableName, + final String storeName) { final String mergeProcessorName = named.suffixWithOrElseGet( "-cogroup-merge", @@ -241,7 +241,7 @@ KTable createTable(final Collection processors, builder.addGraphNode(processors, mergeNode); - return new KTableImpl( + return new KTableImpl( mergeProcessorName, keySerde, valueSerde, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java index 0dacd1345be33..395fdfc234523 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java @@ -68,7 +68,7 @@ class GroupedStreamAggregateBuilder { KTable build(final NamedInternal functionName, final StoreBuilder storeBuilder, - final KStreamAggregateProcessorSupplier aggregateSupplier, + final KStreamAggregateProcessorSupplier aggregateSupplier, final String queryableStoreName, final Serde keySerde, final Serde valueSerde) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java index 0c7d8cc48dcff..d265140162441 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java @@ -235,7 +235,7 @@ public SessionWindowedKStream windowedBy(final SessionWindows windows) { } private KTable doAggregate( - final KStreamAggregateProcessorSupplier aggregateSupplier, + final KStreamAggregateProcessorSupplier aggregateSupplier, final String functionName, final MaterializedInternal> materializedInternal) { return aggregateBuilder.build( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index 6780a51680271..257396f1f438a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -33,24 +33,24 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamAggregate implements KStreamAggregateProcessorSupplier { +public class KStreamAggregate implements KStreamAggregateProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamAggregate.class); private final String storeName; - private final Initializer initializer; - private final Aggregator aggregator; + private final Initializer initializer; + private final Aggregator aggregator; private boolean sendOldValues = false; KStreamAggregate(final String storeName, - final Initializer initializer, - final Aggregator aggregator) { + final Initializer initializer, + final Aggregator aggregator) { this.storeName = storeName; this.initializer = initializer; this.aggregator = aggregator; } @Override - public Processor> get() { + public Processor> get() { return new KStreamAggregateProcessor(); } @@ -60,13 +60,13 @@ public void enableSendingOldValues() { } - private class KStreamAggregateProcessor extends ContextualProcessor> { - private TimestampedKeyValueStore store; + private class KStreamAggregateProcessor extends ContextualProcessor> { + private TimestampedKeyValueStore store; private Sensor droppedRecordsSensor; - private TupleChangeForwarder tupleForwarder; + private TupleChangeForwarder tupleForwarder; @Override - public void init(final ProcessorContext> context) { + public void init(final ProcessorContext> context) { super.init(context); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), @@ -95,10 +95,10 @@ public void process(final Record record) { return; } - final ValueAndTimestamp oldAggAndTimestamp = store.get(record.key()); - T oldAgg = getValueOrNull(oldAggAndTimestamp); + final ValueAndTimestamp oldAggAndTimestamp = store.get(record.key()); + Agg oldAgg = getValueOrNull(oldAggAndTimestamp); - final T newAgg; + final Agg newAgg; final long newTimestamp; if (oldAgg == null) { @@ -117,10 +117,10 @@ public void process(final Record record) { } @Override - public KTableValueGetterSupplier view() { - return new KTableValueGetterSupplier() { + public KTableValueGetterSupplier view() { + return new KTableValueGetterSupplier() { - public KTableValueGetter get() { + public KTableValueGetter get() { return new KStreamAggregateValueGetter(); } @@ -132,8 +132,8 @@ public String[] storeNames() { } - private class KStreamAggregateValueGetter implements KTableValueGetter { - private TimestampedKeyValueStore store; + private class KStreamAggregateValueGetter implements KTableValueGetter { + private TimestampedKeyValueStore store; @Override @@ -142,7 +142,7 @@ public void init(final ProcessorContext con } @Override - public ValueAndTimestamp get(final K key) { + public ValueAndTimestamp get(final K key) { return store.get(key); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java index bd64c8a2293b3..73730d3cab63a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregateProcessorSupplier.java @@ -18,9 +18,9 @@ import org.apache.kafka.streams.processor.api.ProcessorSupplier; -public interface KStreamAggregateProcessorSupplier extends ProcessorSupplier> { +public interface KStreamAggregateProcessorSupplier extends ProcessorSupplier> { - KTableValueGetterSupplier view(); + KTableValueGetterSupplier view(); void enableSendingOldValues(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index 6176bea916bef..d1f4d1da33935 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -32,7 +32,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamReduce implements KStreamAggregateProcessorSupplier { +public class KStreamReduce implements KStreamAggregateProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamReduce.class); private final String storeName; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index a21f687c19f21..cc0deeadd5ee6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -43,7 +43,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor; import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; -public class KStreamSessionWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { +public class KStreamSessionWindowAggregate implements KStreamAggregateProcessorSupplier, Agg> { private static final Logger LOG = LoggerFactory.getLogger(KStreamSessionWindowAggregate.class); private final String storeName; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index b6e3b24f4d73f..80d887ddae514 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -43,7 +43,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamSlidingWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { +public class KStreamSlidingWindowAggregate implements KStreamAggregateProcessorSupplier, Agg> { private final Logger log = LoggerFactory.getLogger(getClass()); private final String storeName; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 6b58bb1f27823..8882c2cb93f1e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -41,7 +41,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -public class KStreamWindowAggregate implements KStreamAggregateProcessorSupplier, V, Agg> { +public class KStreamWindowAggregate implements KStreamAggregateProcessorSupplier, Agg> { private final Logger log = LoggerFactory.getLogger(getClass()); private final String storeName; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 5bd02858da365..8cf58b4b3b40b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -815,7 +815,7 @@ public KTableValueGetterSupplier valueGetterSupplier() { source.materialize(); return new KTableSourceValueGetterSupplier<>(source.queryableName()); } else if (processorSupplier instanceof KStreamAggregateProcessorSupplier) { - return ((KStreamAggregateProcessorSupplier) processorSupplier).view(); + return ((KStreamAggregateProcessorSupplier) processorSupplier).view(); } else { return ((KTableChangeProcessorSupplier) processorSupplier).view(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java index 6224337795bc3..d1be8356acce1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java @@ -26,7 +26,7 @@ import java.util.Collection; public class KTablePassThrough implements KTableChangeProcessorSupplier { - private final Collection parents; //TODO change to aggregationprocessor + private final Collection parents; private final String storeName; From 846f75604b9fd1f7c3e7e842e14e5572217b3e0f Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 15 Apr 2021 09:57:25 +0100 Subject: [PATCH 33/40] rollback change --- .../kstream/internals/KStreamFlatMapValues.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 9197103c9e362..1008b297b3d3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -17,10 +17,10 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ProcessorAdapter; +import org.apache.kafka.streams.processor.api.Record; class KStreamFlatMapValues implements ProcessorSupplier { @@ -32,15 +32,15 @@ class KStreamFlatMapValues implements ProcessorSupplier get() { - return ProcessorAdapter.adaptRaw(new KStreamFlatMapValuesProcessor()); + return new KStreamFlatMapValuesProcessor(); } - private class KStreamFlatMapValuesProcessor extends AbstractProcessor { + private class KStreamFlatMapValuesProcessor extends ContextualProcessor { @Override - public void process(final KIn key, final VIn value) { - final Iterable newValues = mapper.apply(key, value); + public void process(final Record record) { + final Iterable newValues = mapper.apply(record.key(), record.value()); for (final VOut v : newValues) { - context().forward(key, v); + context().forward(record.withValue(v)); } } } From 5057f88622dd4bd4b7f18f5ab01bf24275e4d0e5 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 15 Apr 2021 10:05:03 +0100 Subject: [PATCH 34/40] rm todo comment --- .../kafka/streams/kstream/internals/KStreamWindowAggregate.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 8882c2cb93f1e..978dd0f8f8e64 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -109,7 +109,7 @@ public void init(final ProcessorContext, Change> context) { @Override public void process(final Record record) { - if (record.key() == null) { //TODO + if (record.key() == null) { log.warn( "Skipping record due to null key. value=[{}] topic=[{}] partition=[{}] offset=[{}]", record.value(), From 0f9f4921a27b526c1375b0a6e383a73381629e16 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 15 Apr 2021 10:20:36 +0100 Subject: [PATCH 35/40] align type names --- .../streams/kstream/internals/KTableImpl.java | 238 +++++++++--------- .../internals/KTableKTableAbstractJoin.java | 6 +- .../internals/KTableKTableInnerJoin.java | 2 +- .../internals/KTableKTableLeftJoin.java | 24 +- .../internals/KTableKTableOuterJoin.java | 26 +- .../internals/KTableKTableRightJoin.java | 24 +- 6 files changed, 160 insertions(+), 160 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 8cf58b4b3b40b..dc7d491b4203c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -589,115 +589,115 @@ private SuppressedInternal buildSuppress(final Suppressed suppress } @Override - public KTable join(final KTable other, - final ValueJoiner joiner) { + public KTable join(final KTable other, + final ValueJoiner joiner) { return doJoin(other, joiner, NamedInternal.empty(), null, false, false); } @Override - public KTable join(final KTable other, - final ValueJoiner joiner, - final Named named) { + public KTable join(final KTable other, + final ValueJoiner joiner, + final Named named) { return doJoin(other, joiner, named, null, false, false); } @Override - public KTable join(final KTable other, - final ValueJoiner joiner, - final Materialized> materialized) { + public KTable join(final KTable other, + final ValueJoiner joiner, + final Materialized> materialized) { return join(other, joiner, NamedInternal.empty(), materialized); } @Override - public KTable join(final KTable other, - final ValueJoiner joiner, - final Named named, - final Materialized> materialized) { + public KTable join(final KTable other, + final ValueJoiner joiner, + final Named named, + final Materialized> materialized) { Objects.requireNonNull(materialized, "materialized can't be null"); - final MaterializedInternal> materializedInternal = + final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized, builder, MERGE_NAME); return doJoin(other, joiner, named, materializedInternal, false, false); } @Override - public KTable outerJoin(final KTable other, - final ValueJoiner joiner) { + public KTable outerJoin(final KTable other, + final ValueJoiner joiner) { return outerJoin(other, joiner, NamedInternal.empty()); } @Override - public KTable outerJoin(final KTable other, - final ValueJoiner joiner, - final Named named) { + public KTable outerJoin(final KTable other, + final ValueJoiner joiner, + final Named named) { return doJoin(other, joiner, named, null, true, true); } @Override - public KTable outerJoin(final KTable other, - final ValueJoiner joiner, - final Materialized> materialized) { + public KTable outerJoin(final KTable other, + final ValueJoiner joiner, + final Materialized> materialized) { return outerJoin(other, joiner, NamedInternal.empty(), materialized); } @Override - public KTable outerJoin(final KTable other, - final ValueJoiner joiner, - final Named named, - final Materialized> materialized) { + public KTable outerJoin(final KTable other, + final ValueJoiner joiner, + final Named named, + final Materialized> materialized) { Objects.requireNonNull(materialized, "materialized can't be null"); - final MaterializedInternal> materializedInternal = + final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized, builder, MERGE_NAME); return doJoin(other, joiner, named, materializedInternal, true, true); } @Override - public KTable leftJoin(final KTable other, - final ValueJoiner joiner) { + public KTable leftJoin(final KTable other, + final ValueJoiner joiner) { return leftJoin(other, joiner, NamedInternal.empty()); } @Override - public KTable leftJoin(final KTable other, - final ValueJoiner joiner, - final Named named) { + public KTable leftJoin(final KTable other, + final ValueJoiner joiner, + final Named named) { return doJoin(other, joiner, named, null, true, false); } @Override - public KTable leftJoin(final KTable other, - final ValueJoiner joiner, - final Materialized> materialized) { + public KTable leftJoin(final KTable other, + final ValueJoiner joiner, + final Materialized> materialized) { return leftJoin(other, joiner, NamedInternal.empty(), materialized); } @Override - public KTable leftJoin(final KTable other, - final ValueJoiner joiner, - final Named named, - final Materialized> materialized) { + public KTable leftJoin(final KTable other, + final ValueJoiner joiner, + final Named named, + final Materialized> materialized) { Objects.requireNonNull(materialized, "materialized can't be null"); - final MaterializedInternal> materializedInternal = + final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized, builder, MERGE_NAME); return doJoin(other, joiner, named, materializedInternal, true, false); } @SuppressWarnings("unchecked") - private KTable doJoin(final KTable other, - final ValueJoiner joiner, - final Named joinName, - final MaterializedInternal> materializedInternal, - final boolean leftOuter, - final boolean rightOuter) { + private KTable doJoin(final KTable other, + final ValueJoiner joiner, + final Named joinName, + final MaterializedInternal> materializedInternal, + final boolean leftOuter, + final boolean rightOuter) { Objects.requireNonNull(other, "other can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(joinName, "joinName can't be null"); final NamedInternal renamed = new NamedInternal(joinName); final String joinMergeName = renamed.orElseGenerateWithPrefix(builder, MERGE_NAME); - final Set allSourceNodes = ensureCopartitionWith(Collections.singleton((AbstractStream) other)); + final Set allSourceNodes = ensureCopartitionWith(Collections.singleton((AbstractStream) other)); if (leftOuter) { enableSendingOldValues(true); @@ -706,30 +706,30 @@ private KTable doJoin(final KTable other, ((KTableImpl) other).enableSendingOldValues(true); } - final KTableKTableAbstractJoin joinThis; - final KTableKTableAbstractJoin joinOther; + final KTableKTableAbstractJoin joinThis; + final KTableKTableAbstractJoin joinOther; if (!leftOuter) { // inner - joinThis = new KTableKTableInnerJoin<>(this, (KTableImpl) other, joiner); - joinOther = new KTableKTableInnerJoin<>((KTableImpl) other, this, reverseJoiner(joiner)); + joinThis = new KTableKTableInnerJoin<>(this, (KTableImpl) other, joiner); + joinOther = new KTableKTableInnerJoin<>((KTableImpl) other, this, reverseJoiner(joiner)); } else if (!rightOuter) { // left - joinThis = new KTableKTableLeftJoin<>(this, (KTableImpl) other, joiner); - joinOther = new KTableKTableRightJoin<>((KTableImpl) other, this, reverseJoiner(joiner)); + joinThis = new KTableKTableLeftJoin<>(this, (KTableImpl) other, joiner); + joinOther = new KTableKTableRightJoin<>((KTableImpl) other, this, reverseJoiner(joiner)); } else { // outer - joinThis = new KTableKTableOuterJoin<>(this, (KTableImpl) other, joiner); - joinOther = new KTableKTableOuterJoin<>((KTableImpl) other, this, reverseJoiner(joiner)); + joinThis = new KTableKTableOuterJoin<>(this, (KTableImpl) other, joiner); + joinOther = new KTableKTableOuterJoin<>((KTableImpl) other, this, reverseJoiner(joiner)); } final String joinThisName = renamed.suffixWithOrElseGet("-join-this", builder, JOINTHIS_NAME); final String joinOtherName = renamed.suffixWithOrElseGet("-join-other", builder, JOINOTHER_NAME); final ProcessorParameters, ?, ?> joinThisProcessorParameters = new ProcessorParameters<>(joinThis, joinThisName); - final ProcessorParameters, ?, ?> joinOtherProcessorParameters = new ProcessorParameters<>(joinOther, joinOtherName); + final ProcessorParameters, ?, ?> joinOtherProcessorParameters = new ProcessorParameters<>(joinOther, joinOtherName); final Serde keySerde; - final Serde valueSerde; + final Serde valueSerde; final String queryableStoreName; - final StoreBuilder> storeBuilder; + final StoreBuilder> storeBuilder; if (materializedInternal != null) { if (materializedInternal.keySerde() == null) { @@ -746,8 +746,8 @@ private KTable doJoin(final KTable other, storeBuilder = null; } - final KTableKTableJoinNode kTableKTableJoinNode = - KTableKTableJoinNode.kTableKTableJoinNodeBuilder() + final KTableKTableJoinNode kTableKTableJoinNode = + KTableKTableJoinNode.kTableKTableJoinNodeBuilder() .withNodeName(joinMergeName) .withJoinThisProcessorParameters(joinThisProcessorParameters) .withJoinOtherProcessorParameters(joinOtherProcessorParameters) @@ -764,7 +764,7 @@ private KTable doJoin(final KTable other, builder.addGraphNode(this.graphNode, kTableKTableJoinNode); // we can inherit parent key serde if user do not provide specific overrides - return new KTableImpl, VR>( + return new KTableImpl, VOut>( kTableKTableJoinNode.nodeName(), kTableKTableJoinNode.keySerde(), kTableKTableJoinNode.valueSerde(), @@ -858,9 +858,9 @@ boolean sendingOldValueEnabled() { } @Override - public KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner) { + public KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner) { return doJoinOnForeignKey( other, foreignKeyExtractor, @@ -872,10 +872,10 @@ public KTable join(final KTable other, } @Override - public KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named) { + public KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named) { return doJoinOnForeignKey( other, foreignKeyExtractor, @@ -887,26 +887,26 @@ public KTable join(final KTable other, } @Override - public KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Materialized> materialized) { + public KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Materialized> materialized) { return doJoinOnForeignKey(other, foreignKeyExtractor, joiner, NamedInternal.empty(), materialized, false); } @Override - public KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named, - final Materialized> materialized) { + public KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named, + final Materialized> materialized) { return doJoinOnForeignKey(other, foreignKeyExtractor, joiner, named, materialized, false); } @Override - public KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner) { + public KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner) { return doJoinOnForeignKey( other, foreignKeyExtractor, @@ -918,10 +918,10 @@ public KTable leftJoin(final KTable other, } @Override - public KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named) { + public KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named) { return doJoinOnForeignKey( other, foreignKeyExtractor, @@ -933,29 +933,29 @@ public KTable leftJoin(final KTable other, } @Override - public KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named, - final Materialized> materialized) { + public KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named, + final Materialized> materialized) { return doJoinOnForeignKey(other, foreignKeyExtractor, joiner, named, materialized, true); } @Override - public KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Materialized> materialized) { + public KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Materialized> materialized) { return doJoinOnForeignKey(other, foreignKeyExtractor, joiner, NamedInternal.empty(), materialized, true); } @SuppressWarnings("unchecked") - private KTable doJoinOnForeignKey(final KTable foreignKeyTable, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named joinName, - final Materialized> materialized, - final boolean leftJoin) { + private KTable doJoinOnForeignKey(final KTable foreignKeyTable, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named joinName, + final Materialized> materialized, + final boolean leftJoin) { Objects.requireNonNull(foreignKeyTable, "foreignKeyTable can't be null"); Objects.requireNonNull(foreignKeyExtractor, "foreignKeyExtractor can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); @@ -993,12 +993,12 @@ private KTable doJoinOnForeignKey(final KTable forei builder.internalTopologyBuilder.addInternalTopic(subscriptionTopicName, InternalTopicProperties.empty()); - final Serde foreignKeySerde = ((KTableImpl) foreignKeyTable).keySerde; + final Serde foreignKeySerde = ((KTableImpl) foreignKeyTable).keySerde; final Serde> subscriptionWrapperSerde = new SubscriptionWrapperSerde<>(subscriptionPrimaryKeySerdePseudoTopic, keySerde); - final SubscriptionResponseWrapperSerde responseWrapperSerde = - new SubscriptionResponseWrapperSerde<>(((KTableImpl) foreignKeyTable).valueSerde); + final SubscriptionResponseWrapperSerde responseWrapperSerde = + new SubscriptionResponseWrapperSerde<>(((KTableImpl) foreignKeyTable).valueSerde); - final CombinedKeySchema combinedKeySchema = new CombinedKeySchema<>( + final CombinedKeySchema combinedKeySchema = new CombinedKeySchema<>( subscriptionForeignKeySerdePseudoTopic, foreignKeySerde, subscriptionPrimaryKeySerdePseudoTopic, @@ -1021,14 +1021,14 @@ private KTable doJoinOnForeignKey(final KTable forei builder.addGraphNode(graphNode, subscriptionNode); - final StreamSinkNode> subscriptionSink = new StreamSinkNode<>( + final StreamSinkNode> subscriptionSink = new StreamSinkNode<>( renamed.suffixWithOrElseGet("-subscription-registration-sink", builder, SINK_NAME), new StaticTopicNameExtractor<>(subscriptionTopicName), new ProducedInternal<>(Produced.with(foreignKeySerde, subscriptionWrapperSerde)) ); builder.addGraphNode(subscriptionNode, subscriptionSink); - final StreamSourceNode> subscriptionSource = new StreamSourceNode<>( + final StreamSourceNode> subscriptionSource = new StreamSourceNode<>( renamed.suffixWithOrElseGet("-subscription-registration-source", builder, SOURCE_NAME), Collections.singleton(subscriptionTopicName), new ConsumedInternal<>(Consumed.with(foreignKeySerde, subscriptionWrapperSerde)) @@ -1053,7 +1053,7 @@ private KTable doJoinOnForeignKey(final KTable forei ); builder.addStateStore(subscriptionStore); - final StatefulProcessorNode> subscriptionReceiveNode = + final StatefulProcessorNode> subscriptionReceiveNode = new StatefulProcessorNode<>( new ProcessorParameters<>( new SubscriptionStoreReceiveProcessorSupplier<>(subscriptionStore, combinedKeySchema), @@ -1064,21 +1064,21 @@ private KTable doJoinOnForeignKey(final KTable forei ); builder.addGraphNode(subscriptionSource, subscriptionReceiveNode); - final StatefulProcessorNode, Change>>> subscriptionJoinForeignNode = + final StatefulProcessorNode, Change>>> subscriptionJoinForeignNode = new StatefulProcessorNode<>( new ProcessorParameters<>( new SubscriptionJoinForeignProcessorSupplier<>( - ((KTableImpl) foreignKeyTable).valueGetterSupplier() + ((KTableImpl) foreignKeyTable).valueGetterSupplier() ), renamed.suffixWithOrElseGet("-subscription-join-foreign", builder, SUBSCRIPTION_PROCESSOR) ), Collections.emptySet(), Collections.singleton( - ((KTableImpl) foreignKeyTable).valueGetterSupplier()) + ((KTableImpl) foreignKeyTable).valueGetterSupplier()) ); builder.addGraphNode(subscriptionReceiveNode, subscriptionJoinForeignNode); - final StatefulProcessorNode> foreignJoinSubscriptionNode = new StatefulProcessorNode<>( + final StatefulProcessorNode> foreignJoinSubscriptionNode = new StatefulProcessorNode<>( new ProcessorParameters<>( new ForeignJoinSubscriptionProcessorSupplier<>(subscriptionStore, combinedKeySchema), renamed.suffixWithOrElseGet("-foreign-join-subscription", builder, SUBSCRIPTION_PROCESSOR) @@ -1086,13 +1086,13 @@ private KTable doJoinOnForeignKey(final KTable forei Collections.singleton(subscriptionStore), Collections.emptySet() ); - builder.addGraphNode(((KTableImpl) foreignKeyTable).graphNode, foreignJoinSubscriptionNode); + builder.addGraphNode(((KTableImpl) foreignKeyTable).graphNode, foreignJoinSubscriptionNode); final String finalRepartitionTopicName = renamed.suffixWithOrElseGet("-subscription-response", builder, SUBSCRIPTION_RESPONSE) + TOPIC_SUFFIX; builder.internalTopologyBuilder.addInternalTopic(finalRepartitionTopicName, InternalTopicProperties.empty()); - final StreamSinkNode> foreignResponseSink = + final StreamSinkNode> foreignResponseSink = new StreamSinkNode<>( renamed.suffixWithOrElseGet("-subscription-response-sink", builder, SINK_NAME), new StaticTopicNameExtractor<>(finalRepartitionTopicName), @@ -1101,7 +1101,7 @@ private KTable doJoinOnForeignKey(final KTable forei builder.addGraphNode(subscriptionJoinForeignNode, foreignResponseSink); builder.addGraphNode(foreignJoinSubscriptionNode, foreignResponseSink); - final StreamSourceNode> foreignResponseSource = new StreamSourceNode<>( + final StreamSourceNode> foreignResponseSource = new StreamSourceNode<>( renamed.suffixWithOrElseGet("-subscription-response-source", builder, SOURCE_NAME), Collections.singleton(finalRepartitionTopicName), new ConsumedInternal<>(Consumed.with(keySerde, responseWrapperSerde)) @@ -1114,7 +1114,7 @@ private KTable doJoinOnForeignKey(final KTable forei builder.internalTopologyBuilder.copartitionSources(resultSourceNodes); final KTableValueGetterSupplier primaryKeyValueGetter = valueGetterSupplier(); - final SubscriptionResolverJoinProcessorSupplier resolverProcessorSupplier = + final SubscriptionResolverJoinProcessorSupplier resolverProcessorSupplier = new SubscriptionResolverJoinProcessorSupplier<>( primaryKeyValueGetter, valueSerde == null ? null : valueSerde.serializer(), @@ -1122,7 +1122,7 @@ private KTable doJoinOnForeignKey(final KTable forei joiner, leftJoin ); - final StatefulProcessorNode> resolverNode = new StatefulProcessorNode<>( + final StatefulProcessorNode> resolverNode = new StatefulProcessorNode<>( new ProcessorParameters<>( resolverProcessorSupplier, renamed.suffixWithOrElseGet("-subscription-response-resolver", builder, @@ -1135,7 +1135,7 @@ private KTable doJoinOnForeignKey(final KTable forei final String resultProcessorName = renamed.suffixWithOrElseGet("-result", builder, FK_JOIN_OUTPUT_NAME); - final MaterializedInternal> materializedInternal = + final MaterializedInternal> materializedInternal = new MaterializedInternal<>( materialized, builder, @@ -1148,15 +1148,15 @@ private KTable doJoinOnForeignKey(final KTable forei materializedInternal.withKeySerde(keySerde); } - final KTableSource resultProcessorSupplier = new KTableSource<>( + final KTableSource resultProcessorSupplier = new KTableSource<>( materializedInternal.storeName(), materializedInternal.queryableStoreName() ); - final StoreBuilder> resultStore = + final StoreBuilder> resultStore = new TimestampedKeyValueStoreMaterializer<>(materializedInternal).materialize(); - final TableProcessorNode resultNode = new TableProcessorNode<>( + final TableProcessorNode resultNode = new TableProcessorNode<>( resultProcessorName, new ProcessorParameters<>( resultProcessorSupplier, @@ -1166,7 +1166,7 @@ private KTable doJoinOnForeignKey(final KTable forei ); builder.addGraphNode(resolverNode, resultNode); - return new KTableImpl( + return new KTableImpl( resultProcessorName, keySerde, materializedInternal.valueSerde(), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java index 89ff90a585d33..6d730fade8313 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java @@ -18,19 +18,19 @@ import org.apache.kafka.streams.kstream.ValueJoiner; -abstract class KTableKTableAbstractJoin implements KTableChangeProcessorSupplier { +abstract class KTableKTableAbstractJoin implements KTableChangeProcessorSupplier { private final KTableImpl table1; private final KTableImpl table2; final KTableValueGetterSupplier valueGetterSupplier1; final KTableValueGetterSupplier valueGetterSupplier2; - final ValueJoiner joiner; + final ValueJoiner joiner; boolean sendOldValues = false; KTableKTableAbstractJoin(final KTableImpl table1, final KTableImpl table2, - final ValueJoiner joiner) { + final ValueJoiner joiner) { this.table1 = table1; this.table2 = table2; this.valueGetterSupplier1 = table1.valueGetterSupplier(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 39d211d44198a..cc91c65497666 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -32,7 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class KTableKTableInnerJoin extends KTableKTableAbstractJoin { +class KTableKTableInnerJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableInnerJoin.class); private final KeyValueMapper keyValueMapper = (key, value) -> key; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 661508318bf69..15091f05a541c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -32,39 +32,39 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableKTableLeftJoin extends KTableKTableAbstractJoin { +class KTableKTableLeftJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableLeftJoin.class); KTableKTableLeftJoin(final KTableImpl table1, final KTableImpl table2, - final ValueJoiner joiner) { + final ValueJoiner joiner) { super(table1, table2, joiner); } @Override - public Processor, K, Change> get() { + public Processor, K, Change> get() { return new KTableKTableLeftJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableLeftJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } - private class KTableKTableLeftJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableLeftJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { KTableKTableLeftJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { + public KTableValueGetter get() { return new KTableKTableLeftJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } - private class KTableKTableLeftJoinProcessor extends ContextualProcessor, K, Change> { + private class KTableKTableLeftJoinProcessor extends ContextualProcessor, K, Change> { private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; @@ -75,7 +75,7 @@ private class KTableKTableLeftJoinProcessor extends ContextualProcessor> context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -97,9 +97,9 @@ public void process(final Record> record) { return; } - R newValue = null; + VOut newValue = null; final long resultTimestamp; - R oldValue = null; + VOut oldValue = null; final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(record.key()); final V2 value2 = getValueOrNull(valueAndTimestampRight); @@ -134,7 +134,7 @@ public void close() { } } - private class KTableKTableLeftJoinValueGetter implements KTableValueGetter { + private class KTableKTableLeftJoinValueGetter implements KTableValueGetter { private final KTableValueGetter valueGetter1; private final KTableValueGetter valueGetter2; @@ -152,7 +152,7 @@ public void init(final ProcessorContext con } @Override - public ValueAndTimestamp get(final K key) { + public ValueAndTimestamp get(final K key) { final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); final V1 value1 = getValueOrNull(valueAndTimestamp1); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 65af581dd37fb..c35e4fd2a578b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -32,38 +32,38 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableKTableOuterJoin extends KTableKTableAbstractJoin { +class KTableKTableOuterJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableOuterJoin.class); KTableKTableOuterJoin(final KTableImpl table1, final KTableImpl table2, - final ValueJoiner joiner) { + final ValueJoiner joiner) { super(table1, table2, joiner); } @Override - public Processor, K, Change> get() { + public Processor, K, Change> get() { return new KTableKTableOuterJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableOuterJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } - private class KTableKTableOuterJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableOuterJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { KTableKTableOuterJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { + public KTableValueGetter get() { return new KTableKTableOuterJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } - private class KTableKTableOuterJoinProcessor extends ContextualProcessor, K, Change> { + private class KTableKTableOuterJoinProcessor extends ContextualProcessor, K, Change> { private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; @@ -74,7 +74,7 @@ private class KTableKTableOuterJoinProcessor extends ContextualProcessor> context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -96,9 +96,9 @@ public void process(final Record> record) { return; } - R newValue = null; + VOut newValue = null; final long resultTimestamp; - R oldValue = null; + VOut oldValue = null; final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(record.key()); final V2 value2 = getValueOrNull(valueAndTimestamp2); @@ -129,7 +129,7 @@ public void close() { } } - private class KTableKTableOuterJoinValueGetter implements KTableValueGetter { + private class KTableKTableOuterJoinValueGetter implements KTableValueGetter { private final KTableValueGetter valueGetter1; private final KTableValueGetter valueGetter2; @@ -147,8 +147,8 @@ public void init(final ProcessorContext con } @Override - public ValueAndTimestamp get(final K key) { - R newValue = null; + public ValueAndTimestamp get(final K key) { + VOut newValue = null; final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); final V1 value1; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index 3c9b5ac3d50d0..37b44a9f08915 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -31,38 +31,38 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableKTableRightJoin extends KTableKTableAbstractJoin { +class KTableKTableRightJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableRightJoin.class); KTableKTableRightJoin(final KTableImpl table1, final KTableImpl table2, - final ValueJoiner joiner) { + final ValueJoiner joiner) { super(table1, table2, joiner); } @Override - public Processor, K, Change> get() { + public Processor, K, Change> get() { return new KTableKTableRightJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableRightJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } - private class KTableKTableRightJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableRightJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { KTableKTableRightJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { + public KTableValueGetter get() { return new KTableKTableRightJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } - private class KTableKTableRightJoinProcessor extends ContextualProcessor, K, Change> { + private class KTableKTableRightJoinProcessor extends ContextualProcessor, K, Change> { private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; @@ -73,7 +73,7 @@ private class KTableKTableRightJoinProcessor extends ContextualProcessor> context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -95,9 +95,9 @@ public void process(final Record> record) { return; } - final R newValue; + final VOut newValue; final long resultTimestamp; - R oldValue = null; + VOut oldValue = null; final ValueAndTimestamp valueAndTimestampLeft = valueGetter.get(record.key()); final V2 valueLeft = getValueOrNull(valueAndTimestampLeft); @@ -124,7 +124,7 @@ public void close() { } } - private class KTableKTableRightJoinValueGetter implements KTableValueGetter { + private class KTableKTableRightJoinValueGetter implements KTableValueGetter { private final KTableValueGetter valueGetter1; private final KTableValueGetter valueGetter2; @@ -142,7 +142,7 @@ public void init(final ProcessorContext con } @Override - public ValueAndTimestamp get(final K key) { + public ValueAndTimestamp get(final K key) { final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); final V2 value2 = getValueOrNull(valueAndTimestamp2); From 585070551d186ef4bcf2f455ac4e5d6a23e4e2b8 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 15 Apr 2021 14:14:04 +0100 Subject: [PATCH 36/40] align type names --- .../apache/kafka/streams/kstream/KStream.java | 4113 +++++++++-------- .../apache/kafka/streams/kstream/KTable.java | 441 +- .../kafka/streams/kstream/ValueJoiner.java | 10 +- .../streams/kstream/ValueJoinerWithKey.java | 12 +- .../kstream/internals/AbstractStream.java | 2 +- .../internals/KStreamGlobalKTableJoin.java | 16 +- .../kstream/internals/KStreamImpl.java | 320 +- .../kstream/internals/KStreamImplJoin.java | 46 +- .../kstream/internals/KStreamKStreamJoin.java | 18 +- .../kstream/internals/KStreamKTableJoin.java | 14 +- .../internals/KStreamKTableJoinProcessor.java | 22 +- .../KTableChangeProcessorSupplier.java | 4 +- .../streams/kstream/internals/KTableImpl.java | 96 +- .../internals/KTableKTableAbstractJoin.java | 30 +- ...KTableAbstractJoinValueGetterSupplier.java | 12 +- .../internals/KTableKTableInnerJoin.java | 62 +- .../internals/KTableKTableLeftJoin.java | 46 +- .../internals/KTableKTableOuterJoin.java | 44 +- .../internals/KTableKTableRightJoin.java | 46 +- .../internals/StreamJoinedInternal.java | 8 +- .../graph/BaseJoinProcessorNode.java | 26 +- .../internals/graph/KTableKTableJoinNode.java | 74 +- .../internals/graph/StreamStreamJoinNode.java | 76 +- .../kstream/internals/KTableImplTest.java | 3 +- 24 files changed, 2921 insertions(+), 2620 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index afbaf03d23da0..4b82e54ab00ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -26,29 +26,29 @@ import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.processor.ConnectedStoreProvider; import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopicNameExtractor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; /** - * {@code KStream} is an abstraction of a record stream of {@link KeyValue} pairs, i.e., each record is an - * independent entity/event in the real world. - * For example a user X might buy two items I1 and I2, and thus there might be two records {@code , } - * in the stream. + * {@code KStream} is an abstraction of a record stream of {@link KeyValue} pairs, i.e., each + * record is an independent entity/event in the real world. For example a user X might buy two items + * I1 and I2, and thus there might be two records {@code , } in the stream. *

- * A {@code KStream} is either {@link StreamsBuilder#stream(String) defined from one or multiple Kafka topics} that - * are consumed message by message or the result of a {@code KStream} transformation. - * A {@link KTable} can also be {@link KTable#toStream() converted} into a {@code KStream}. + * A {@code KStream} is either {@link StreamsBuilder#stream(String) defined from one or multiple + * Kafka topics} that are consumed message by message or the result of a {@code KStream} + * transformation. A {@link KTable} can also be {@link KTable#toStream() converted} into a {@code + * KStream}. *

- * A {@code KStream} can be transformed record by record, joined with another {@code KStream}, {@link KTable}, - * {@link GlobalKTable}, or can be aggregated into a {@link KTable}. - * Kafka Streams DSL can be mixed-and-matched with Processor API (PAPI) (c.f. {@link Topology}) via - * {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) process(...)}, - * {@link #transform(TransformerSupplier, String...) transform(...)}, and - * {@link #transformValues(ValueTransformerSupplier, String...) transformValues(...)}. + * A {@code KStream} can be transformed record by record, joined with another {@code KStream}, + * {@link KTable}, {@link GlobalKTable}, or can be aggregated into a {@link KTable}. Kafka Streams + * DSL can be mixed-and-matched with Processor API (PAPI) (c.f. {@link Topology}) via {@link + * #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) process(...)}, {@link + * #transform(TransformerSupplier, String...) transform(...)}, and {@link + * #transformValues(ValueTransformerSupplier, String...) transformValues(...)}. * * @param Type of keys * @param Type of values @@ -59,9 +59,9 @@ public interface KStream { /** - * Create a new {@code KStream} that consists of all records of this stream which satisfy the given predicate. - * All records that do not satisfy the predicate are dropped. - * This is a stateless record-by-record operation. + * Create a new {@code KStream} that consists of all records of this stream which satisfy the + * given predicate. All records that do not satisfy the predicate are dropped. This is a + * stateless record-by-record operation. * * @param predicate a filter {@link Predicate} that is applied to each record * @return a {@code KStream} that contains only those records that satisfy the given predicate @@ -70,9 +70,9 @@ public interface KStream { KStream filter(final Predicate predicate); /** - * Create a new {@code KStream} that consists of all records of this stream which satisfy the given predicate. - * All records that do not satisfy the predicate are dropped. - * This is a stateless record-by-record operation. + * Create a new {@code KStream} that consists of all records of this stream which satisfy the + * given predicate. All records that do not satisfy the predicate are dropped. This is a + * stateless record-by-record operation. * * @param predicate a filter {@link Predicate} that is applied to each record * @param named a {@link Named} config used to name the processor in the topology @@ -82,39 +82,39 @@ public interface KStream { KStream filter(final Predicate predicate, final Named named); /** - * Create a new {@code KStream} that consists all records of this stream which do not satisfy the given - * predicate. - * All records that do satisfy the predicate are dropped. + * Create a new {@code KStream} that consists all records of this stream which do not + * satisfy the given predicate. All records that do satisfy the predicate are dropped. * This is a stateless record-by-record operation. * * @param predicate a filter {@link Predicate} that is applied to each record - * @return a {@code KStream} that contains only those records that do not satisfy the given predicate + * @return a {@code KStream} that contains only those records that do not satisfy the + * given predicate * @see #filter(Predicate) */ KStream filterNot(final Predicate predicate); /** - * Create a new {@code KStream} that consists all records of this stream which do not satisfy the given - * predicate. - * All records that do satisfy the predicate are dropped. + * Create a new {@code KStream} that consists all records of this stream which do not + * satisfy the given predicate. All records that do satisfy the predicate are dropped. * This is a stateless record-by-record operation. * * @param predicate a filter {@link Predicate} that is applied to each record * @param named a {@link Named} config used to name the processor in the topology - * @return a {@code KStream} that contains only those records that do not satisfy the given predicate + * @return a {@code KStream} that contains only those records that do not satisfy the + * given predicate * @see #filter(Predicate) */ KStream filterNot(final Predicate predicate, final Named named); /** - * Set a new key (with possibly new type) for each input record. - * The provided {@link KeyValueMapper} is applied to each input record and computes a new key for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation. - *

- * For example, you can use this transformation to set a key for a key-less input record {@code } by - * extracting a key from the value within your {@link KeyValueMapper}. The example below computes the new key as the - * length of the value string. + * Set a new key (with possibly new type) for each input record. The provided {@link + * KeyValueMapper} is applied to each input record and computes a new key for it. Thus, an input + * record {@code } can be transformed into an output record {@code }. This is a + * stateless record-by-record operation. + *

+ * For example, you can use this transformation to set a key for a key-less input record {@code + * } by extracting a key from the value within your {@link KeyValueMapper}. The example + * below computes the new key as the length of the value string. *

{@code
      * KStream keyLessStream = builder.stream("key-less-topic");
      * KStream keyedStream = keyLessStream.selectKey(new KeyValueMapper {
@@ -123,12 +123,13 @@ public interface KStream {
      *     }
      * });
      * }
- * Setting a new key might result in an internal data redistribution if a key based operator (like an aggregation or - * join) is applied to the result {@code KStream}. + * Setting a new key might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. * * @param mapper a {@link KeyValueMapper} that computes a new key for each record - * @param the new key type of the result stream - * @return a {@code KStream} that contains records with new key (possibly of different type) and unmodified value + * @param the new key type of the result stream + * @return a {@code KStream} that contains records with new key (possibly of different type) and + * unmodified value * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #mapValues(ValueMapper) @@ -136,17 +137,18 @@ public interface KStream { * @see #flatMapValues(ValueMapper) * @see #flatMapValues(ValueMapperWithKey) */ - KStream selectKey(final KeyValueMapper mapper); + KStream selectKey( + final KeyValueMapper mapper); /** - * Set a new key (with possibly new type) for each input record. - * The provided {@link KeyValueMapper} is applied to each input record and computes a new key for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation. - *

- * For example, you can use this transformation to set a key for a key-less input record {@code } by - * extracting a key from the value within your {@link KeyValueMapper}. The example below computes the new key as the - * length of the value string. + * Set a new key (with possibly new type) for each input record. The provided {@link + * KeyValueMapper} is applied to each input record and computes a new key for it. Thus, an input + * record {@code } can be transformed into an output record {@code }. This is a + * stateless record-by-record operation. + *

+ * For example, you can use this transformation to set a key for a key-less input record {@code + * } by extracting a key from the value within your {@link KeyValueMapper}. The example + * below computes the new key as the length of the value string. *

{@code
      * KStream keyLessStream = builder.stream("key-less-topic");
      * KStream keyedStream = keyLessStream.selectKey(new KeyValueMapper {
@@ -155,13 +157,14 @@ public interface KStream {
      *     }
      * });
      * }
- * Setting a new key might result in an internal data redistribution if a key based operator (like an aggregation or - * join) is applied to the result {@code KStream}. + * Setting a new key might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. * * @param mapper a {@link KeyValueMapper} that computes a new key for each record * @param named a {@link Named} config used to name the processor in the topology - * @param the new key type of the result stream - * @return a {@code KStream} that contains records with new key (possibly of different type) and unmodified value + * @param the new key type of the result stream + * @return a {@code KStream} that contains records with new key (possibly of different type) and + * unmodified value * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #mapValues(ValueMapper) @@ -169,18 +172,20 @@ public interface KStream { * @see #flatMapValues(ValueMapper) * @see #flatMapValues(ValueMapperWithKey) */ - KStream selectKey(final KeyValueMapper mapper, - final Named named); + KStream selectKey( + final KeyValueMapper mapper, + final Named named); /** - * Transform each record of the input stream into a new record in the output stream (both key and value type can be - * altered arbitrarily). - * The provided {@link KeyValueMapper} is applied to each input record and computes a new output record. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, String...)} for - * stateful record transformation). - *

- * The example below normalizes the String key to upper-case letters and counts the number of token of the value string. + * Transform each record of the input stream into a new record in the output stream (both key + * and value type can be altered arbitrarily). The provided {@link KeyValueMapper} is applied to + * each input record and computes a new output record. Thus, an input record {@code } can + * be transformed into an output record {@code }. This is a stateless record-by-record + * operation (cf. {@link #transform(TransformerSupplier, String...)} for stateful record + * transformation). + *

+ * The example below normalizes the String key to upper-case letters and counts the number of + * token of the value string. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.map(new KeyValueMapper> {
@@ -189,15 +194,18 @@  KStream selectKey(final KeyValueMapper
-     * The provided {@link KeyValueMapper} must return a {@link KeyValue} type and must not return {@code null}.
+     * The provided {@link KeyValueMapper} must return a {@link KeyValue} type and must not return
+     * {@code null}.
      * 

- * Mapping records might result in an internal data redistribution if a key based operator (like an aggregation or - * join) is applied to the result {@code KStream}. (cf. {@link #mapValues(ValueMapper)}) + * Mapping records might result in an internal data redistribution if a key based operator (like + * an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #mapValues(ValueMapper)}) * * @param mapper a {@link KeyValueMapper} that computes a new output record - * @param the key type of the result stream - * @param the value type of the result stream - * @return a {@code KStream} that contains records with new key and value (possibly both of different type) + * @param the key type of the result stream + * @param the value type of the result stream + * @return a {@code KStream} that contains records with new key and value (possibly both of + * different type) * @see #selectKey(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #mapValues(ValueMapper) @@ -208,17 +216,19 @@ KStream selectKey(final KeyValueMapper KStream map(final KeyValueMapper> mapper); + KStream map( + final KeyValueMapper> mapper); /** - * Transform each record of the input stream into a new record in the output stream (both key and value type can be - * altered arbitrarily). - * The provided {@link KeyValueMapper} is applied to each input record and computes a new output record. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, String...)} for - * stateful record transformation). - *

- * The example below normalizes the String key to upper-case letters and counts the number of token of the value string. + * Transform each record of the input stream into a new record in the output stream (both key + * and value type can be altered arbitrarily). The provided {@link KeyValueMapper} is applied to + * each input record and computes a new output record. Thus, an input record {@code } can + * be transformed into an output record {@code }. This is a stateless record-by-record + * operation (cf. {@link #transform(TransformerSupplier, String...)} for stateful record + * transformation). + *

+ * The example below normalizes the String key to upper-case letters and counts the number of + * token of the value string. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.map(new KeyValueMapper> {
@@ -227,16 +237,19 @@  KStream selectKey(final KeyValueMapper
-     * The provided {@link KeyValueMapper} must return a {@link KeyValue} type and must not return {@code null}.
+     * The provided {@link KeyValueMapper} must return a {@link KeyValue} type and must not return
+     * {@code null}.
      * 

- * Mapping records might result in an internal data redistribution if a key based operator (like an aggregation or - * join) is applied to the result {@code KStream}. (cf. {@link #mapValues(ValueMapper)}) + * Mapping records might result in an internal data redistribution if a key based operator (like + * an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #mapValues(ValueMapper)}) * * @param mapper a {@link KeyValueMapper} that computes a new output record * @param named a {@link Named} config used to name the processor in the topology - * @param the key type of the result stream - * @param the value type of the result stream - * @return a {@code KStream} that contains records with new key and value (possibly both of different type) + * @param the key type of the result stream + * @param the value type of the result stream + * @return a {@code KStream} that contains records with new key and value (possibly both of + * different type) * @see #selectKey(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #mapValues(ValueMapper) @@ -247,15 +260,16 @@ KStream selectKey(final KeyValueMapper KStream map(final KeyValueMapper> mapper, - final Named named); + KStream map( + final KeyValueMapper> mapper, + final Named named); /** - * Transform the value of each input record into a new value (with possible new type) of the output record. - * The provided {@link ValueMapper} is applied to each input record value and computes a new value for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation (cf. - * {@link #transformValues(ValueTransformerSupplier, String...)} for stateful value transformation). + * Transform the value of each input record into a new value (with possible new type) of the + * output record. The provided {@link ValueMapper} is applied to each input record value and + * computes a new value for it. Thus, an input record {@code } can be transformed into an + * output record {@code }. This is a stateless record-by-record operation (cf. {@link + * #transformValues(ValueTransformerSupplier, String...)} for stateful value transformation). *

* The example below counts the number of token of the value string. *

{@code
@@ -266,13 +280,14 @@  KStream map(final KeyValueMapper
-     * Setting a new value preserves data co-location with respect to the key.
-     * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join)
-     * is applied to the result {@code KStream}. (cf. {@link #map(KeyValueMapper)})
+     * Setting a new value preserves data co-location with respect to the key. Thus, no
+     * internal data redistribution is required if a key based operator (like an aggregation or
+     * join) is applied to the result {@code KStream}. (cf. {@link #map(KeyValueMapper)})
      *
      * @param mapper a {@link ValueMapper} that computes a new output value
-     * @param    the value type of the result stream
-     * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type)
+     * @param  the value type of the result stream
+     * @return a {@code KStream} that contains records with unmodified key and new values (possibly
+     * of different type)
      * @see #selectKey(KeyValueMapper)
      * @see #map(KeyValueMapper)
      * @see #flatMap(KeyValueMapper)
@@ -282,14 +297,14 @@  KStream map(final KeyValueMapper KStream mapValues(final ValueMapper mapper);
+     KStream mapValues(final ValueMapper mapper);
 
     /**
-     * Transform the value of each input record into a new value (with possible new type) of the output record.
-     * The provided {@link ValueMapper} is applied to each input record value and computes a new value for it.
-     * Thus, an input record {@code } can be transformed into an output record {@code }.
-     * This is a stateless record-by-record operation (cf.
-     * {@link #transformValues(ValueTransformerSupplier, String...)} for stateful value transformation).
+     * Transform the value of each input record into a new value (with possible new type) of the
+     * output record. The provided {@link ValueMapper} is applied to each input record value and
+     * computes a new value for it. Thus, an input record {@code } can be transformed into an
+     * output record {@code }. This is a stateless record-by-record operation (cf. {@link
+     * #transformValues(ValueTransformerSupplier, String...)} for stateful value transformation).
      * 

* The example below counts the number of token of the value string. *

{@code
@@ -300,14 +315,15 @@  KStream map(final KeyValueMapper
-     * Setting a new value preserves data co-location with respect to the key.
-     * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join)
-     * is applied to the result {@code KStream}. (cf. {@link #map(KeyValueMapper)})
+     * Setting a new value preserves data co-location with respect to the key. Thus, no
+     * internal data redistribution is required if a key based operator (like an aggregation or
+     * join) is applied to the result {@code KStream}. (cf. {@link #map(KeyValueMapper)})
      *
      * @param mapper a {@link ValueMapper} that computes a new output value
      * @param named  a {@link Named} config used to name the processor in the topology
-     * @param    the value type of the result stream
-     * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type)
+     * @param  the value type of the result stream
+     * @return a {@code KStream} that contains records with unmodified key and new values (possibly
+     * of different type)
      * @see #selectKey(KeyValueMapper)
      * @see #map(KeyValueMapper)
      * @see #flatMap(KeyValueMapper)
@@ -317,15 +333,16 @@  KStream map(final KeyValueMapper KStream mapValues(final ValueMapper mapper,
-                                  final Named named);
+     KStream mapValues(final ValueMapper mapper,
+                                      final Named named);
 
     /**
-     * Transform the value of each input record into a new value (with possible new type) of the output record.
-     * The provided {@link ValueMapperWithKey} is applied to each input record value and computes a new value for it.
-     * Thus, an input record {@code } can be transformed into an output record {@code }.
-     * This is a stateless record-by-record operation (cf.
-     * {@link #transformValues(ValueTransformerWithKeySupplier, String...)} for stateful value transformation).
+     * Transform the value of each input record into a new value (with possible new type) of the
+     * output record. The provided {@link ValueMapperWithKey} is applied to each input record value
+     * and computes a new value for it. Thus, an input record {@code } can be transformed into
+     * an output record {@code }. This is a stateless record-by-record operation (cf. {@link
+     * #transformValues(ValueTransformerWithKeySupplier, String...)} for stateful value
+     * transformation).
      * 

* The example below counts the number of tokens of key and value strings. *

{@code
@@ -336,14 +353,16 @@  KStream mapValues(final ValueMapper mapper,
      *     }
      * });
      * }
- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #map(KeyValueMapper)}) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, setting a new value preserves data co-location with respect to the key. + * Thus, no internal data redistribution is required if a key based operator (like an + * aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #map(KeyValueMapper)}) * * @param mapper a {@link ValueMapperWithKey} that computes a new output value - * @param the value type of the result stream - * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type) + * @param the value type of the result stream + * @return a {@code KStream} that contains records with unmodified key and new values (possibly + * of different type) * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) @@ -353,14 +372,16 @@ KStream mapValues(final ValueMapper mapper, * @see #transformValues(ValueTransformerSupplier, String...) * @see #transformValues(ValueTransformerWithKeySupplier, String...) */ - KStream mapValues(final ValueMapperWithKey mapper); + KStream mapValues( + final ValueMapperWithKey mapper); /** - * Transform the value of each input record into a new value (with possible new type) of the output record. - * The provided {@link ValueMapperWithKey} is applied to each input record value and computes a new value for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation (cf. - * {@link #transformValues(ValueTransformerWithKeySupplier, String...)} for stateful value transformation). + * Transform the value of each input record into a new value (with possible new type) of the + * output record. The provided {@link ValueMapperWithKey} is applied to each input record value + * and computes a new value for it. Thus, an input record {@code } can be transformed into + * an output record {@code }. This is a stateless record-by-record operation (cf. {@link + * #transformValues(ValueTransformerWithKeySupplier, String...)} for stateful value + * transformation). *

* The example below counts the number of tokens of key and value strings. *

{@code
@@ -371,15 +392,17 @@  KStream mapValues(final ValueMapper mapper,
      *     }
      * });
      * }
- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #map(KeyValueMapper)}) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, setting a new value preserves data co-location with respect to the key. + * Thus, no internal data redistribution is required if a key based operator (like an + * aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #map(KeyValueMapper)}) * * @param mapper a {@link ValueMapperWithKey} that computes a new output value * @param named a {@link Named} config used to name the processor in the topology - * @param the value type of the result stream - * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type) + * @param the value type of the result stream + * @return a {@code KStream} that contains records with unmodified key and new values (possibly + * of different type) * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) @@ -389,19 +412,20 @@ KStream mapValues(final ValueMapper mapper, * @see #transformValues(ValueTransformerSupplier, String...) * @see #transformValues(ValueTransformerWithKeySupplier, String...) */ - KStream mapValues(final ValueMapperWithKey mapper, - final Named named); + KStream mapValues( + final ValueMapperWithKey mapper, + final Named named); /** - * Transform each record of the input stream into zero or more records in the output stream (both key and value type - * can be altered arbitrarily). - * The provided {@link KeyValueMapper} is applied to each input record and computes zero or more output records. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, String...)} for - * stateful record transformation). - *

- * The example below splits input records {@code } containing sentences as values into their words - * and emit a record {@code } for each word. + * Transform each record of the input stream into zero or more records in the output stream + * (both key and value type can be altered arbitrarily). The provided {@link KeyValueMapper} is + * applied to each input record and computes zero or more output records. Thus, an input record + * {@code } can be transformed into output records {@code , , ...}. This is + * a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, + * String...)} for stateful record transformation). + *

+ * The example below splits input records {@code } containing sentences as values + * into their words and emit a record {@code } for each word. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.flatMap(
@@ -418,16 +442,18 @@  KStream mapValues(final ValueMapperWithKey
-     * The provided {@link KeyValueMapper} must return an {@link Iterable} (e.g., any {@link java.util.Collection} type)
-     * and the return value must not be {@code null}.
+     * The provided {@link KeyValueMapper} must return an {@link Iterable} (e.g., any {@link
+     * java.util.Collection} type) and the return value must not be {@code null}.
      * 

- * Flat-mapping records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@code KStream}. (cf. {@link #flatMapValues(ValueMapper)}) + * Flat-mapping records might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #flatMapValues(ValueMapper)}) * * @param mapper a {@link KeyValueMapper} that computes the new output records - * @param the key type of the result stream - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) + * @param the key type of the result stream + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with new key and value (possibly + * of different type) * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #mapValues(ValueMapper) @@ -441,18 +467,19 @@ KStream mapValues(final ValueMapperWithKey KStream flatMap(final KeyValueMapper>> mapper); + KStream flatMap( + final KeyValueMapper>> mapper); /** - * Transform each record of the input stream into zero or more records in the output stream (both key and value type - * can be altered arbitrarily). - * The provided {@link KeyValueMapper} is applied to each input record and computes zero or more output records. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, String...)} for - * stateful record transformation). - *

- * The example below splits input records {@code } containing sentences as values into their words - * and emit a record {@code } for each word. + * Transform each record of the input stream into zero or more records in the output stream + * (both key and value type can be altered arbitrarily). The provided {@link KeyValueMapper} is + * applied to each input record and computes zero or more output records. Thus, an input record + * {@code } can be transformed into output records {@code , , ...}. This is + * a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, + * String...)} for stateful record transformation). + *

+ * The example below splits input records {@code } containing sentences as values + * into their words and emit a record {@code } for each word. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.flatMap(
@@ -469,17 +496,19 @@  KStream mapValues(final ValueMapperWithKey
-     * The provided {@link KeyValueMapper} must return an {@link Iterable} (e.g., any {@link java.util.Collection} type)
-     * and the return value must not be {@code null}.
+     * The provided {@link KeyValueMapper} must return an {@link Iterable} (e.g., any {@link
+     * java.util.Collection} type) and the return value must not be {@code null}.
      * 

- * Flat-mapping records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@code KStream}. (cf. {@link #flatMapValues(ValueMapper)}) + * Flat-mapping records might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #flatMapValues(ValueMapper)}) * * @param mapper a {@link KeyValueMapper} that computes the new output records * @param named a {@link Named} config used to name the processor in the topology - * @param the key type of the result stream - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) + * @param the key type of the result stream + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with new key and value (possibly + * of different type) * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #mapValues(ValueMapper) @@ -493,20 +522,22 @@ KStream mapValues(final ValueMapperWithKey KStream flatMap(final KeyValueMapper>> mapper, - final Named named); + KStream flatMap( + final KeyValueMapper>> mapper, + final Named named); /** - * Create a new {@code KStream} by transforming the value of each record in this stream into zero or more values - * with the same key in the new stream. - * Transform the value of each input record into zero or more records with the same (unmodified) key in the output - * stream (value type can be altered arbitrarily). - * The provided {@link ValueMapper} is applied to each input record and computes zero or more output values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerSupplier, String...)} + * Create a new {@code KStream} by transforming the value of each record in this stream into + * zero or more values with the same key in the new stream. Transform the value of each input + * record into zero or more records with the same (unmodified) key in the output stream (value + * type can be altered arbitrarily). The provided {@link ValueMapper} is applied to each input + * record and computes zero or more output values. Thus, an input record {@code } can be + * transformed into output records {@code , , ...}. This is a stateless + * record-by-record operation (cf. {@link #transformValues(ValueTransformerSupplier, String...)} * for stateful value transformation). *

- * The example below splits input records {@code } containing sentences as values into their words. + * The example below splits input records {@code } containing sentences as values + * into their words. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.flatMapValues(new ValueMapper> {
@@ -515,16 +546,18 @@  KStream flatMap(final KeyValueMapper
-     * The provided {@link ValueMapper} must return an {@link Iterable} (e.g., any {@link java.util.Collection} type)
-     * and the return value must not be {@code null}.
+     * The provided {@link ValueMapper} must return an {@link Iterable} (e.g., any {@link
+     * java.util.Collection} type) and the return value must not be {@code null}.
      * 

- * Splitting a record into multiple records with the same key preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatMap(KeyValueMapper)}) + * Splitting a record into multiple records with the same key preserves data co-location with + * respect to the key. Thus, no internal data redistribution is required if a key based + * operator (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #flatMap(KeyValueMapper)}) * * @param mapper a {@link ValueMapper} the computes the new output values - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified keys and new values of different type + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified keys and new + * values of different type * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) @@ -537,19 +570,21 @@ KStream flatMap(final KeyValueMapper KStream flatMapValues(final ValueMapper> mapper); + KStream flatMapValues( + final ValueMapper> mapper); /** - * Create a new {@code KStream} by transforming the value of each record in this stream into zero or more values - * with the same key in the new stream. - * Transform the value of each input record into zero or more records with the same (unmodified) key in the output - * stream (value type can be altered arbitrarily). - * The provided {@link ValueMapper} is applied to each input record and computes zero or more output values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerSupplier, String...)} + * Create a new {@code KStream} by transforming the value of each record in this stream into + * zero or more values with the same key in the new stream. Transform the value of each input + * record into zero or more records with the same (unmodified) key in the output stream (value + * type can be altered arbitrarily). The provided {@link ValueMapper} is applied to each input + * record and computes zero or more output values. Thus, an input record {@code } can be + * transformed into output records {@code , , ...}. This is a stateless + * record-by-record operation (cf. {@link #transformValues(ValueTransformerSupplier, String...)} * for stateful value transformation). *

- * The example below splits input records {@code } containing sentences as values into their words. + * The example below splits input records {@code } containing sentences as values + * into their words. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.flatMapValues(new ValueMapper> {
@@ -558,17 +593,19 @@  KStream flatMap(final KeyValueMapper
-     * The provided {@link ValueMapper} must return an {@link Iterable} (e.g., any {@link java.util.Collection} type)
-     * and the return value must not be {@code null}.
+     * The provided {@link ValueMapper} must return an {@link Iterable} (e.g., any {@link
+     * java.util.Collection} type) and the return value must not be {@code null}.
      * 

- * Splitting a record into multiple records with the same key preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatMap(KeyValueMapper)}) + * Splitting a record into multiple records with the same key preserves data co-location with + * respect to the key. Thus, no internal data redistribution is required if a key based + * operator (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #flatMap(KeyValueMapper)}) * * @param mapper a {@link ValueMapper} the computes the new output values * @param named a {@link Named} config used to name the processor in the topology - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified keys and new values of different type + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified keys and new + * values of different type * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) @@ -581,20 +618,22 @@ KStream flatMap(final KeyValueMapper KStream flatMapValues(final ValueMapper> mapper, - final Named named); + KStream flatMapValues( + final ValueMapper> mapper, + final Named named); + /** - * Create a new {@code KStream} by transforming the value of each record in this stream into zero or more values - * with the same key in the new stream. - * Transform the value of each input record into zero or more records with the same (unmodified) key in the output - * stream (value type can be altered arbitrarily). - * The provided {@link ValueMapperWithKey} is applied to each input record and computes zero or more output values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerWithKeySupplier, String...)} - * for stateful value transformation). - *

- * The example below splits input records {@code }, with key=1, containing sentences as values - * into their words. + * Create a new {@code KStream} by transforming the value of each record in this stream into + * zero or more values with the same key in the new stream. Transform the value of each input + * record into zero or more records with the same (unmodified) key in the output stream (value + * type can be altered arbitrarily). The provided {@link ValueMapperWithKey} is applied to each + * input record and computes zero or more output values. Thus, an input record {@code } can + * be transformed into output records {@code , , ...}. This is a stateless + * record-by-record operation (cf. {@link #transformValues(ValueTransformerWithKeySupplier, + * String...)} for stateful value transformation). + *

+ * The example below splits input records {@code }, with key=1, containing + * sentences as values into their words. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.flatMapValues(new ValueMapper> {
@@ -607,17 +646,19 @@  KStream flatMapValues(final ValueMapper
-     * The provided {@link ValueMapperWithKey} must return an {@link Iterable} (e.g., any {@link java.util.Collection} type)
-     * and the return value must not be {@code null}.
+     * The provided {@link ValueMapperWithKey} must return an {@link Iterable} (e.g., any {@link
+     * java.util.Collection} type) and the return value must not be {@code null}.
      * 

- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, splitting a record into multiple records with the same key preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatMap(KeyValueMapper)}) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, splitting a record into multiple records with the same key preserves data + * co-location with respect to the key. Thus, no internal data redistribution is + * required if a key based operator (like an aggregation or join) is applied to the result + * {@code KStream}. (cf. {@link #flatMap(KeyValueMapper)}) * * @param mapper a {@link ValueMapperWithKey} the computes the new output values - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified keys and new values of different type + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified keys and new + * values of different type * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) @@ -630,20 +671,21 @@ KStream flatMapValues(final ValueMapper KStream flatMapValues(final ValueMapperWithKey> mapper); + KStream flatMapValues( + final ValueMapperWithKey> mapper); /** - * Create a new {@code KStream} by transforming the value of each record in this stream into zero or more values - * with the same key in the new stream. - * Transform the value of each input record into zero or more records with the same (unmodified) key in the output - * stream (value type can be altered arbitrarily). - * The provided {@link ValueMapperWithKey} is applied to each input record and computes zero or more output values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerWithKeySupplier, String...)} - * for stateful value transformation). - *

- * The example below splits input records {@code }, with key=1, containing sentences as values - * into their words. + * Create a new {@code KStream} by transforming the value of each record in this stream into + * zero or more values with the same key in the new stream. Transform the value of each input + * record into zero or more records with the same (unmodified) key in the output stream (value + * type can be altered arbitrarily). The provided {@link ValueMapperWithKey} is applied to each + * input record and computes zero or more output values. Thus, an input record {@code } can + * be transformed into output records {@code , , ...}. This is a stateless + * record-by-record operation (cf. {@link #transformValues(ValueTransformerWithKeySupplier, + * String...)} for stateful value transformation). + *

+ * The example below splits input records {@code }, with key=1, containing + * sentences as values into their words. *

{@code
      * KStream inputStream = builder.stream("topic");
      * KStream outputStream = inputStream.flatMapValues(new ValueMapper> {
@@ -656,18 +698,20 @@  KStream flatMapValues(final ValueMapper
-     * The provided {@link ValueMapperWithKey} must return an {@link Iterable} (e.g., any {@link java.util.Collection} type)
-     * and the return value must not be {@code null}.
+     * The provided {@link ValueMapperWithKey} must return an {@link Iterable} (e.g., any {@link
+     * java.util.Collection} type) and the return value must not be {@code null}.
      * 

- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, splitting a record into multiple records with the same key preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatMap(KeyValueMapper)}) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, splitting a record into multiple records with the same key preserves data + * co-location with respect to the key. Thus, no internal data redistribution is + * required if a key based operator (like an aggregation or join) is applied to the result + * {@code KStream}. (cf. {@link #flatMap(KeyValueMapper)}) * * @param mapper a {@link ValueMapperWithKey} the computes the new output values * @param named a {@link Named} config used to name the processor in the topology - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified keys and new values of different type + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified keys and new + * values of different type * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) @@ -680,22 +724,24 @@ KStream flatMapValues(final ValueMapper KStream flatMapValues(final ValueMapperWithKey> mapper, - final Named named); + KStream flatMapValues( + final ValueMapperWithKey> mapper, + final Named named); /** - * Print the records of this KStream using the options provided by {@link Printed} - * Note that this is mainly for debugging/testing purposes, and it will try to flush on each record print. - * It SHOULD NOT be used for production usage if performance requirements are concerned. + * Print the records of this KStream using the options provided by {@link Printed} Note that + * this is mainly for debugging/testing purposes, and it will try to flush on each record print. + * It SHOULD NOT be used for production usage if performance requirements are + * concerned. * * @param printed options for printing */ void print(final Printed printed); /** - * Perform an action on each record of {@code KStream}. - * This is a stateless record-by-record operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...)}). - * Note that this is a terminal operation that returns void. + * Perform an action on each record of {@code KStream}. This is a stateless record-by-record + * operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, + * String...)}). Note that this is a terminal operation that returns void. * * @param action an action to perform on each record * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) @@ -703,9 +749,9 @@ KStream flatMapValues(final ValueMapperWithKey action); /** - * Perform an action on each record of {@code KStream}. - * This is a stateless record-by-record operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...)}). - * Note that this is a terminal operation that returns void. + * Perform an action on each record of {@code KStream}. This is a stateless record-by-record + * operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, + * String...)}). Note that this is a terminal operation that returns void. * * @param action an action to perform on each record * @param named a {@link Named} config used to name the processor in the topology @@ -714,45 +760,49 @@ KStream flatMapValues(final ValueMapperWithKey action, final Named named); /** - * Perform an action on each record of {@code KStream}. - * This is a stateless record-by-record operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...)}). + * Perform an action on each record of {@code KStream}. This is a stateless record-by-record + * operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, + * String...)}). *

- * Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection) - * and returns an unchanged stream. + * Peek is a non-terminal operation that triggers a side effect (such as logging or statistics + * collection) and returns an unchanged stream. *

- * Note that since this operation is stateless, it may execute multiple times for a single record in failure cases. + * Note that since this operation is stateless, it may execute multiple times for a single + * record in failure cases. * * @param action an action to perform on each record - * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) * @return itself + * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ KStream peek(final ForeachAction action); /** - * Perform an action on each record of {@code KStream}. - * This is a stateless record-by-record operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...)}). + * Perform an action on each record of {@code KStream}. This is a stateless record-by-record + * operation (cf. {@link #process(org.apache.kafka.streams.processor.ProcessorSupplier, + * String...)}). *

- * Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection) - * and returns an unchanged stream. + * Peek is a non-terminal operation that triggers a side effect (such as logging or statistics + * collection) and returns an unchanged stream. *

- * Note that since this operation is stateless, it may execute multiple times for a single record in failure cases. + * Note that since this operation is stateless, it may execute multiple times for a single + * record in failure cases. * * @param action an action to perform on each record * @param named a {@link Named} config used to name the processor in the topology - * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) * @return itself + * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ KStream peek(final ForeachAction action, final Named named); /** - * Creates an array of {@code KStream} from this stream by branching the records in the original stream based on - * the supplied predicates. - * Each record is evaluated against the supplied predicates, and predicates are evaluated in order. - * Each stream in the result array corresponds position-wise (index) to the predicate in the supplied predicates. - * The branching happens on first-match: A record in the original stream is assigned to the corresponding result - * stream for the first predicate that evaluates to true, and is assigned to this stream only. - * A record will be dropped if none of the predicates evaluate to true. - * This is a stateless record-by-record operation. + * Creates an array of {@code KStream} from this stream by branching the records in the original + * stream based on the supplied predicates. Each record is evaluated against the supplied + * predicates, and predicates are evaluated in order. Each stream in the result array + * corresponds position-wise (index) to the predicate in the supplied predicates. The branching + * happens on first-match: A record in the original stream is assigned to the corresponding + * result stream for the first predicate that evaluates to true, and is assigned to this stream + * only. A record will be dropped if none of the predicates evaluate to true. This is a + * stateless record-by-record operation. * * @param predicates the ordered list of {@link Predicate} instances * @return multiple distinct substreams of this {@code KStream} @@ -763,16 +813,16 @@ KStream flatMapValues(final ValueMapperWithKey[] branch(final Predicate... predicates); /** - * Creates an array of {@code KStream} from this stream by branching the records in the original stream based on - * the supplied predicates. - * Each record is evaluated against the supplied predicates, and predicates are evaluated in order. - * Each stream in the result array corresponds position-wise (index) to the predicate in the supplied predicates. - * The branching happens on first-match: A record in the original stream is assigned to the corresponding result - * stream for the first predicate that evaluates to true, and is assigned to this stream only. - * A record will be dropped if none of the predicates evaluate to true. - * This is a stateless record-by-record operation. - * - * @param named a {@link Named} config used to name the processor in the topology + * Creates an array of {@code KStream} from this stream by branching the records in the original + * stream based on the supplied predicates. Each record is evaluated against the supplied + * predicates, and predicates are evaluated in order. Each stream in the result array + * corresponds position-wise (index) to the predicate in the supplied predicates. The branching + * happens on first-match: A record in the original stream is assigned to the corresponding + * result stream for the first predicate that evaluates to true, and is assigned to this stream + * only. A record will be dropped if none of the predicates evaluate to true. This is a + * stateless record-by-record operation. + * + * @param named a {@link Named} config used to name the processor in the topology * @param predicates the ordered list of {@link Predicate} instances * @return multiple distinct substreams of this {@code KStream} * @deprecated since 2.8. Use {@link #split(Named)} instead. @@ -782,32 +832,33 @@ KStream flatMapValues(final ValueMapperWithKey[] branch(final Named named, final Predicate... predicates); /** - * Split this stream. {@link BranchedKStream} can be used for routing the records to different branches depending - * on evaluation against the supplied predicates. - * Stream branching is a stateless record-by-record operation. + * Split this stream. {@link BranchedKStream} can be used for routing the records to different + * branches depending on evaluation against the supplied predicates. Stream branching is a + * stateless record-by-record operation. * - * @return {@link BranchedKStream} that provides methods for routing the records to different branches. + * @return {@link BranchedKStream} that provides methods for routing the records to different + * branches. */ BranchedKStream split(); /** - * Split this stream. {@link BranchedKStream} can be used for routing the records to different branches depending - * on evaluation against the supplied predicates. - * Stream branching is a stateless record-by-record operation. - * - * @param named a {@link Named} config used to name the processor in the topology and also to set the name prefix - * for the resulting branches (see {@link BranchedKStream}) - * @return {@link BranchedKStream} that provides methods for routing the records to different branches. + * Split this stream. {@link BranchedKStream} can be used for routing the records to different + * branches depending on evaluation against the supplied predicates. Stream branching is a + * stateless record-by-record operation. + * + * @param named a {@link Named} config used to name the processor in the topology and also to + * set the name prefix for the resulting branches (see {@link BranchedKStream}) + * @return {@link BranchedKStream} that provides methods for routing the records to different + * branches. */ BranchedKStream split(final Named named); /** * Merge this stream and the given stream into one larger stream. *

- * There is no ordering guarantee between records from this {@code KStream} and records from - * the provided {@code KStream} in the merged stream. - * Relative order is preserved within each input stream though (ie, records within one input - * stream are processed in order). + * There is no ordering guarantee between records from this {@code KStream} and records from the + * provided {@code KStream} in the merged stream. Relative order is preserved within each input + * stream though (ie, records within one input stream are processed in order). * * @param stream a stream which is to be merged into this stream * @return a merged stream containing all records from this and the provided {@code KStream} @@ -817,10 +868,9 @@ KStream flatMapValues(final ValueMapperWithKey - * There is no ordering guarantee between records from this {@code KStream} and records from - * the provided {@code KStream} in the merged stream. - * Relative order is preserved within each input stream though (ie, records within one input - * stream are processed in order). + * There is no ordering guarantee between records from this {@code KStream} and records from the + * provided {@code KStream} in the merged stream. Relative order is preserved within each input + * stream though (ie, records within one input stream are processed in order). * * @param stream a stream which is to be merged into this stream * @param named a {@link Named} config used to name the processor in the topology @@ -829,18 +879,20 @@ KStream flatMapValues(final ValueMapperWithKey merge(final KStream stream, final Named named); /** - * Materialize this stream to a topic and creates a new {@code KStream} from the topic using default serializers, - * deserializers, and producer's {@link DefaultPartitioner}. - * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is - * started). + * Materialize this stream to a topic and creates a new {@code KStream} from the topic using + * default serializers, deserializers, and producer's {@link DefaultPartitioner}. The specified + * topic should be manually created before it is used (i.e., before the Kafka Streams + * application is started). *

- * This is similar to calling {@link #to(String) #to(someTopicName)} and - * {@link StreamsBuilder#stream(String) StreamsBuilder#stream(someTopicName)}. - * Note that {@code through()} uses a hard coded {@link org.apache.kafka.streams.processor.FailOnInvalidTimestamp - * timestamp extractor} and does not allow to customize it, to ensure correct timestamp propagation. + * This is similar to calling {@link #to(String) #to(someTopicName)} and {@link + * StreamsBuilder#stream(String) StreamsBuilder#stream(someTopicName)}. Note that {@code + * through()} uses a hard coded {@link org.apache.kafka.streams.processor.FailOnInvalidTimestamp + * timestamp extractor} and does not allow to customize it, to ensure correct timestamp + * propagation. * * @param topic the topic name - * @return a {@code KStream} that contains the exact same (and potentially repartitioned) records as this {@code KStream} + * @return a {@code KStream} that contains the exact same (and potentially repartitioned) + * records as this {@code KStream} * @deprecated since 2.6; use {@link #repartition()} instead */ // TODO: when removed, update `StreamsResetter` decription of --intermediate-topics @@ -849,19 +901,21 @@ KStream flatMapValues(final ValueMapperWithKey - * This is similar to calling {@link #to(String, Produced) to(someTopic, Produced.with(keySerde, valueSerde)} - * and {@link StreamsBuilder#stream(String, Consumed) StreamsBuilder#stream(someTopicName, Consumed.with(keySerde, valueSerde))}. - * Note that {@code through()} uses a hard coded {@link org.apache.kafka.streams.processor.FailOnInvalidTimestamp - * timestamp extractor} and does not allow to customize it, to ensure correct timestamp propagation. - * - * @param topic the topic name - * @param produced the options to use when producing to the topic - * @return a {@code KStream} that contains the exact same (and potentially repartitioned) records as this {@code KStream} + * {@link Produced} instance for configuration of the {@link Serde key serde}, {@link Serde + * value serde}, and {@link StreamPartitioner}. The specified topic should be manually created + * before it is used (i.e., before the Kafka Streams application is started). + *

+ * This is similar to calling {@link #to(String, Produced) to(someTopic, Produced.with(keySerde, + * valueSerde)} and {@link StreamsBuilder#stream(String, Consumed) + * StreamsBuilder#stream(someTopicName, Consumed.with(keySerde, valueSerde))}. Note that {@code + * through()} uses a hard coded {@link org.apache.kafka.streams.processor.FailOnInvalidTimestamp + * timestamp extractor} and does not allow to customize it, to ensure correct timestamp + * propagation. + * + * @param topic the topic name + * @param produced the options to use when producing to the topic + * @return a {@code KStream} that contains the exact same (and potentially repartitioned) + * records as this {@code KStream} * @deprecated since 2.6; use {@link #repartition(Repartitioned)} instead */ @Deprecated @@ -869,75 +923,86 @@ KStream through(final String topic, final Produced produced); /** - * Materialize this stream to an auto-generated repartition topic and create a new {@code KStream} - * from the auto-generated topic using default serializers, deserializers, and producer's {@link DefaultPartitioner}. - * The number of partitions is determined based on the upstream topics partition numbers. - *

- * The created topic is considered as an internal topic and is meant to be used only by the current Kafka Streams instance. - * Similar to auto-repartitioning, the topic will be created with infinite retention time and data will be automatically purged by Kafka Streams. - * The topic will be named as "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. - * - * @return {@code KStream} that contains the exact same repartitioned records as this {@code KStream}. + * Materialize this stream to an auto-generated repartition topic and create a new {@code + * KStream} from the auto-generated topic using default serializers, deserializers, and + * producer's {@link DefaultPartitioner}. The number of partitions is determined based on the + * upstream topics partition numbers. + *

+ * The created topic is considered as an internal topic and is meant to be used only by the + * current Kafka Streams instance. Similar to auto-repartitioning, the topic will be created + * with infinite retention time and data will be automatically purged by Kafka Streams. The + * topic will be named as "${applicationId}-<name>-repartition", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG + * APPLICATION_ID_CONFIG}, "<name>" is an internally generated name, and "-repartition" is + * a fixed suffix. + * + * @return {@code KStream} that contains the exact same repartitioned records as this {@code + * KStream}. */ KStream repartition(); /** - * Materialize this stream to an auto-generated repartition topic and create a new {@code KStream} - * from the auto-generated topic using {@link Serde key serde}, {@link Serde value serde}, {@link StreamPartitioner}, - * number of partitions, and topic name part as defined by {@link Repartitioned}. - *

- * The created topic is considered as an internal topic and is meant to be used only by the current Kafka Streams instance. - * Similar to auto-repartitioning, the topic will be created with infinite retention time and data will be automatically purged by Kafka Streams. - * The topic will be named as "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is either provided via {@link Repartitioned#as(String)} or an internally - * generated name, and "-repartition" is a fixed suffix. + * Materialize this stream to an auto-generated repartition topic and create a new {@code + * KStream} from the auto-generated topic using {@link Serde key serde}, {@link Serde value + * serde}, {@link StreamPartitioner}, number of partitions, and topic name part as defined by + * {@link Repartitioned}. + *

+ * The created topic is considered as an internal topic and is meant to be used only by the + * current Kafka Streams instance. Similar to auto-repartitioning, the topic will be created + * with infinite retention time and data will be automatically purged by Kafka Streams. The + * topic will be named as "${applicationId}-<name>-repartition", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG + * APPLICATION_ID_CONFIG}, "<name>" is either provided via {@link + * Repartitioned#as(String)} or an internally generated name, and "-repartition" is a fixed + * suffix. * * @param repartitioned the {@link Repartitioned} instance used to specify {@link Serdes}, - * {@link StreamPartitioner} which determines how records are distributed among partitions of the topic, - * part of the topic name, and number of partitions for a repartition topic. - * @return a {@code KStream} that contains the exact same repartitioned records as this {@code KStream}. + * {@link StreamPartitioner} which determines how records are distributed + * among partitions of the topic, part of the topic name, and number of + * partitions for a repartition topic. + * @return a {@code KStream} that contains the exact same repartitioned records as this {@code + * KStream}. */ KStream repartition(final Repartitioned repartitioned); /** - * Materialize this stream to a topic using default serializers specified in the config and producer's - * {@link DefaultPartitioner}. - * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is - * started). + * Materialize this stream to a topic using default serializers specified in the config and + * producer's {@link DefaultPartitioner}. The specified topic should be manually created before + * it is used (i.e., before the Kafka Streams application is started). * * @param topic the topic name */ void to(final String topic); /** - * Materialize this stream to a topic using the provided {@link Produced} instance. - * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is - * started). + * Materialize this stream to a topic using the provided {@link Produced} instance. The + * specified topic should be manually created before it is used (i.e., before the Kafka Streams + * application is started). * - * @param topic the topic name - * @param produced the options to use when producing to the topic + * @param topic the topic name + * @param produced the options to use when producing to the topic */ void to(final String topic, final Produced produced); /** - * Dynamically materialize this stream to topics using default serializers specified in the config and producer's - * {@link DefaultPartitioner}. - * The topic names for each record to send to is dynamically determined based on the {@link TopicNameExtractor}. + * Dynamically materialize this stream to topics using default serializers specified in the + * config and producer's {@link DefaultPartitioner}. The topic names for each record to send to + * is dynamically determined based on the {@link TopicNameExtractor}. * - * @param topicExtractor the extractor to determine the name of the Kafka topic to write to for each record + * @param topicExtractor the extractor to determine the name of the Kafka topic to write to for + * each record */ void to(final TopicNameExtractor topicExtractor); /** * Dynamically materialize this stream to topics using the provided {@link Produced} instance. - * The topic names for each record to send to is dynamically determined based on the {@link TopicNameExtractor}. + * The topic names for each record to send to is dynamically determined based on the {@link + * TopicNameExtractor}. * - * @param topicExtractor the extractor to determine the name of the Kafka topic to write to for each record - * @param produced the options to use when producing to the topic + * @param topicExtractor the extractor to determine the name of the Kafka topic to write to for + * each record + * @param produced the options to use when producing to the topic */ void to(final TopicNameExtractor topicExtractor, final Produced produced); @@ -945,24 +1010,26 @@ void to(final TopicNameExtractor topicExtractor, /** * Convert this stream to a {@link KTable}. *

- * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} or - * {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will be created in Kafka. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. + * If a key changing operator was used before this operation (e.g., {@link + * #selectKey(KeyValueMapper)}, {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} + * or {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will + * be created in Kafka. This topic will be named "${applicationId}-<name>-repartition", + * where "applicationId" is user-specified in {@link StreamsConfig} via parameter {@link + * StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is an internally + * generated name, and "-repartition" is a fixed suffix. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the resulting {@link KTable} is partitioned - * correctly on its key. - * Note that you cannot enable {@link StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because - * repartition topics are considered transient and don't allow to recover the result {@link KTable} in cause of - * a failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. + * For this case, all data of this stream will be redistributed through the repartitioning topic + * by writing all records to it, and rereading all records from it, such that the resulting + * {@link KTable} is partitioned correctly on its key. Note that you cannot enable {@link + * StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because repartition topics + * are considered transient and don't allow to recover the result {@link KTable} in cause of a + * failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. *

- * Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of - * it was a "fact/event" and is re-interpreted as update now (cf. {@link KStream} vs {@code KTable}). + * Note that this is a logical operation and only changes the "interpretation" of the stream, + * i.e., each record of it was a "fact/event" and is re-interpreted as update now (cf. {@link + * KStream} vs {@code KTable}). * * @return a {@link KTable} that contains the same records as this {@code KStream} */ @@ -971,26 +1038,28 @@ void to(final TopicNameExtractor topicExtractor, /** * Convert this stream to a {@link KTable}. *

- * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} or - * {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will be created in Kafka. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. + * If a key changing operator was used before this operation (e.g., {@link + * #selectKey(KeyValueMapper)}, {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} + * or {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will + * be created in Kafka. This topic will be named "${applicationId}-<name>-repartition", + * where "applicationId" is user-specified in {@link StreamsConfig} via parameter {@link + * StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is an internally + * generated name, and "-repartition" is a fixed suffix. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the resulting {@link KTable} is partitioned - * correctly on its key. - * Note that you cannot enable {@link StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because - * repartition topics are considered transient and don't allow to recover the result {@link KTable} in cause of - * a failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. + * For this case, all data of this stream will be redistributed through the repartitioning topic + * by writing all records to it, and rereading all records from it, such that the resulting + * {@link KTable} is partitioned correctly on its key. Note that you cannot enable {@link + * StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because repartition topics + * are considered transient and don't allow to recover the result {@link KTable} in cause of a + * failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. *

- * Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of - * it was a "fact/event" and is re-interpreted as update now (cf. {@link KStream} vs {@code KTable}). + * Note that this is a logical operation and only changes the "interpretation" of the stream, + * i.e., each record of it was a "fact/event" and is re-interpreted as update now (cf. {@link + * KStream} vs {@code KTable}). * - * @param named a {@link Named} config used to name the processor in the topology + * @param named a {@link Named} config used to name the processor in the topology * @return a {@link KTable} that contains the same records as this {@code KStream} */ KTable toTable(final Named named); @@ -998,27 +1067,29 @@ void to(final TopicNameExtractor topicExtractor, /** * Convert this stream to a {@link KTable}. *

- * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} or - * {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will be created in Kafka. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. + * If a key changing operator was used before this operation (e.g., {@link + * #selectKey(KeyValueMapper)}, {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} + * or {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will + * be created in Kafka. This topic will be named "${applicationId}-<name>-repartition", + * where "applicationId" is user-specified in {@link StreamsConfig} via parameter {@link + * StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is an internally + * generated name, and "-repartition" is a fixed suffix. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the resulting {@link KTable} is partitioned - * correctly on its key. - * Note that you cannot enable {@link StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because - * repartition topics are considered transient and don't allow to recover the result {@link KTable} in cause of - * a failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. + * For this case, all data of this stream will be redistributed through the repartitioning topic + * by writing all records to it, and rereading all records from it, such that the resulting + * {@link KTable} is partitioned correctly on its key. Note that you cannot enable {@link + * StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because repartition topics + * are considered transient and don't allow to recover the result {@link KTable} in cause of a + * failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. *

- * Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of - * it was a "fact/event" and is re-interpreted as update now (cf. {@link KStream} vs {@code KTable}). + * Note that this is a logical operation and only changes the "interpretation" of the stream, + * i.e., each record of it was a "fact/event" and is re-interpreted as update now (cf. {@link + * KStream} vs {@code KTable}). * - * @param materialized an instance of {@link Materialized} used to describe how the state store of the - * resulting table should be materialized. + * @param materialized an instance of {@link Materialized} used to describe how the state store + * of the resulting table should be materialized. * @return a {@link KTable} that contains the same records as this {@code KStream} */ KTable toTable(final Materialized> materialized); @@ -1026,167 +1097,182 @@ void to(final TopicNameExtractor topicExtractor, /** * Convert this stream to a {@link KTable}. *

- * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} or - * {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will be created in Kafka. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. + * If a key changing operator was used before this operation (e.g., {@link + * #selectKey(KeyValueMapper)}, {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} + * or {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic will + * be created in Kafka. This topic will be named "${applicationId}-<name>-repartition", + * where "applicationId" is user-specified in {@link StreamsConfig} via parameter {@link + * StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is an internally + * generated name, and "-repartition" is a fixed suffix. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the resulting {@link KTable} is partitioned - * correctly on its key. - * Note that you cannot enable {@link StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because - * repartition topics are considered transient and don't allow to recover the result {@link KTable} in cause of - * a failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. + * For this case, all data of this stream will be redistributed through the repartitioning topic + * by writing all records to it, and rereading all records from it, such that the resulting + * {@link KTable} is partitioned correctly on its key. Note that you cannot enable {@link + * StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG} config for this case, because repartition topics + * are considered transient and don't allow to recover the result {@link KTable} in cause of a + * failure; hence, a dedicated changelog topic is required to guarantee fault-tolerance. *

- * Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of - * it was a "fact/event" and is re-interpreted as update now (cf. {@link KStream} vs {@code KTable}). + * Note that this is a logical operation and only changes the "interpretation" of the stream, + * i.e., each record of it was a "fact/event" and is re-interpreted as update now (cf. {@link + * KStream} vs {@code KTable}). * - * @param named a {@link Named} config used to name the processor in the topology - * @param materialized an instance of {@link Materialized} used to describe how the state store of the - * resulting table should be materialized. + * @param named a {@link Named} config used to name the processor in the topology + * @param materialized an instance of {@link Materialized} used to describe how the state store + * of the resulting table should be materialized. * @return a {@link KTable} that contains the same records as this {@code KStream} */ KTable toTable(final Named named, final Materialized> materialized); /** - * Group the records of this {@code KStream} on a new key that is selected using the provided {@link KeyValueMapper} - * and default serializers and deserializers. - * {@link KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. - * Grouping a stream on the record key is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedStream}). - * The {@link KeyValueMapper} selects a new key (which may or may not be of the same type) while preserving the - * original values. - * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedStream} - *

- * Because a new key is selected, an internal repartitioning topic may need to be created in Kafka if a - * later operator depends on the newly selected key. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. + * Group the records of this {@code KStream} on a new key that is selected using the provided + * {@link KeyValueMapper} and default serializers and deserializers. {@link KGroupedStream} can + * be further grouped with other streams to form a {@link CogroupedKStream}. Grouping a stream + * on the record key is required before an aggregation operator can be applied to the data (cf. + * {@link KGroupedStream}). The {@link KeyValueMapper} selects a new key (which may or may not + * be of the same type) while preserving the original values. If the new record key is {@code + * null} the record will not be included in the resulting {@link KGroupedStream} + *

+ * Because a new key is selected, an internal repartitioning topic may need to be created in + * Kafka if a later operator depends on the newly selected key. This topic will be named + * "${applicationId}-<name>-repartition", where "applicationId" is user-specified in + * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG + * APPLICATION_ID_CONFIG}, "<name>" is an internally generated name, and "-repartition" is + * a fixed suffix. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * All data of this stream will be redistributed through the repartitioning topic by writing all records to it, - * and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned on the new key. + * All data of this stream will be redistributed through the repartitioning topic by writing all + * records to it, and rereading all records from it, such that the resulting {@link + * KGroupedStream} is partitioned on the new key. *

- * This operation is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link #groupByKey()}. - * If the key type is changed, it is recommended to use {@link #groupBy(KeyValueMapper, Grouped)} instead. + * This operation is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link + * #groupByKey()}. If the key type is changed, it is recommended to use {@link + * #groupBy(KeyValueMapper, Grouped)} instead. * * @param keySelector a {@link KeyValueMapper} that computes a new key for grouping - * @param the key type of the result {@link KGroupedStream} - * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} + * @param the key type of the result {@link KGroupedStream} + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code + * KStream} */ - KGroupedStream groupBy(final KeyValueMapper keySelector); + KGroupedStream groupBy( + final KeyValueMapper keySelector); /** - * Group the records of this {@code KStream} on a new key that is selected using the provided {@link KeyValueMapper} - * and {@link Serde}s as specified by {@link Grouped}. - * {@link KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. - * Grouping a stream on the record key is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedStream}). - * The {@link KeyValueMapper} selects a new key (which may or may not be of the same type) while preserving the - * original values. - * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. - *

- * Because a new key is selected, an internal repartitioning topic may need to be created in Kafka if a later - * operator depends on the newly selected key. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is either provided via {@link org.apache.kafka.streams.kstream.Grouped#as(String)} or an - * internally generated name. + * Group the records of this {@code KStream} on a new key that is selected using the provided + * {@link KeyValueMapper} and {@link Serde}s as specified by {@link Grouped}. {@link + * KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. + * Grouping a stream on the record key is required before an aggregation operator can be applied + * to the data (cf. {@link KGroupedStream}). The {@link KeyValueMapper} selects a new key (which + * may or may not be of the same type) while preserving the original values. If the new record + * key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. + *

+ * Because a new key is selected, an internal repartitioning topic may need to be created in + * Kafka if a later operator depends on the newly selected key. This topic will be named + * "${applicationId}-<name>-repartition", where "applicationId" is user-specified in + * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG + * APPLICATION_ID_CONFIG}, "<name>" is either provided via {@link + * org.apache.kafka.streams.kstream.Grouped#as(String)} or an internally generated name. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * All data of this stream will be redistributed through the repartitioning topic by writing all records to it, - * and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned on the new key. + * All data of this stream will be redistributed through the repartitioning topic by writing all + * records to it, and rereading all records from it, such that the resulting {@link + * KGroupedStream} is partitioned on the new key. *

- * This operation is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link #groupByKey()}. + * This operation is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link + * #groupByKey()}. * * @param keySelector a {@link KeyValueMapper} that computes a new key for grouping - * @param grouped the {@link Grouped} instance used to specify {@link org.apache.kafka.common.serialization.Serdes} - * and part of the name for a repartition topic if repartitioning is required. - * @param the key type of the result {@link KGroupedStream} - * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} + * @param grouped the {@link Grouped} instance used to specify {@link + * org.apache.kafka.common.serialization.Serdes} and part of the name for a + * repartition topic if repartitioning is required. + * @param the key type of the result {@link KGroupedStream} + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code + * KStream} */ - KGroupedStream groupBy(final KeyValueMapper keySelector, - final Grouped grouped); + KGroupedStream groupBy( + final KeyValueMapper keySelector, + final Grouped grouped); /** - * Group the records by their current key into a {@link KGroupedStream} while preserving the original values - * and default serializers and deserializers. - * {@link KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. - * Grouping a stream on the record key is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedStream}). - * If a record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. - *

- * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} or - * {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic may need to be created in - * Kafka if a later operator depends on the newly selected key. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. + * Group the records by their current key into a {@link KGroupedStream} while preserving the + * original values and default serializers and deserializers. {@link KGroupedStream} can be + * further grouped with other streams to form a {@link CogroupedKStream}. Grouping a stream on + * the record key is required before an aggregation operator can be applied to the data (cf. + * {@link KGroupedStream}). If a record key is {@code null} the record will not be included in + * the resulting {@link KGroupedStream}. + *

+ * If a key changing operator was used before this operation (e.g., {@link + * #selectKey(KeyValueMapper)}, {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} + * or {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic may + * need to be created in Kafka if a later operator depends on the newly selected key. This topic + * will be named "${applicationId}-<name>-repartition", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG + * APPLICATION_ID_CONFIG}, "<name>" is an internally generated name, and "-repartition" is + * a fixed suffix. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned - * correctly on its key. - * If the last key changing operator changed the key type, it is recommended to use - * {@link #groupByKey(org.apache.kafka.streams.kstream.Grouped)} instead. + * For this case, all data of this stream will be redistributed through the repartitioning topic + * by writing all records to it, and rereading all records from it, such that the resulting + * {@link KGroupedStream} is partitioned correctly on its key. If the last key changing operator + * changed the key type, it is recommended to use {@link #groupByKey(org.apache.kafka.streams.kstream.Grouped)} + * instead. * - * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code + * KStream} * @see #groupBy(KeyValueMapper) */ KGroupedStream groupByKey(); /** - * Group the records by their current key into a {@link KGroupedStream} while preserving the original values - * and using the serializers as defined by {@link Grouped}. - * {@link KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. - * Grouping a stream on the record key is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedStream}). - * If a record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. - *

- * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} or - * {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic may need to be created in - * Kafka if a later operator depends on the newly selected key. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * <name> is either provided via {@link org.apache.kafka.streams.kstream.Grouped#as(String)} or an internally - * generated name, and "-repartition" is a fixed suffix. + * Group the records by their current key into a {@link KGroupedStream} while preserving the + * original values and using the serializers as defined by {@link Grouped}. {@link + * KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. + * Grouping a stream on the record key is required before an aggregation operator can be applied + * to the data (cf. {@link KGroupedStream}). If a record key is {@code null} the record will not + * be included in the resulting {@link KGroupedStream}. + *

+ * If a key changing operator was used before this operation (e.g., {@link + * #selectKey(KeyValueMapper)}, {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} + * or {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic may + * need to be created in Kafka if a later operator depends on the newly selected key. This topic + * will be named "${applicationId}-<name>-repartition", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG + * APPLICATION_ID_CONFIG}, <name> is either provided via {@link + * org.apache.kafka.streams.kstream.Grouped#as(String)} or an internally generated name, and + * "-repartition" is a fixed suffix. *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. *

- * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned - * correctly on its key. + * For this case, all data of this stream will be redistributed through the repartitioning topic + * by writing all records to it, and rereading all records from it, such that the resulting + * {@link KGroupedStream} is partitioned correctly on its key. * - * @param grouped the {@link Grouped} instance used to specify {@link Serdes} - * and part of the name for a repartition topic if repartitioning is required. - * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} + * @param grouped the {@link Grouped} instance used to specify {@link Serdes} and part of the + * name for a repartition topic if repartitioning is required. + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code + * KStream} * @see #groupBy(KeyValueMapper) */ KGroupedStream groupByKey(final Grouped grouped); /** - * Join records of this stream with another {@code KStream}'s records using windowed inner equi join with default - * serializers and deserializers. - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed inner equi + * join with default serializers and deserializers. The join is computed on the records' key + * with join attribute {@code thisKStream.key == otherKStream.key}. Furthermore, two records are + * only joined if their timestamps are close to each other as defined by the given {@link + * JoinWindows}, i.e., the window defines an additional join predicate on the record + * timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will + * be called to compute a value (with arbitrary type) for the result record. The key of the + * result record is the same as for both joining input records. If an input record key or value + * is {@code null} the record will not be included in the join operation and thus no output + * record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): * @@ -1238,31 +1324,34 @@ KGroupedStream groupBy(final KeyValueMapper the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key and within the + * joining window intervals * @see #leftJoin(KStream, ValueJoiner, JoinWindows) * @see #outerJoin(KStream, ValueJoiner, JoinWindows) */ - KStream join(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows); + KStream join(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows); /** - * Join records of this stream with another {@code KStream}'s records using windowed inner equi join with default - * serializers and deserializers. - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} will be called to compute - * a value (with arbitrary type) for the result record. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * The key of the result record is the same as for both joining input records. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no + * Join records of this stream with another {@code KStream}'s records using windowed inner equi + * join with default serializers and deserializers. The join is computed on the records' key + * with join attribute {@code thisKStream.key == otherKStream.key}. Furthermore, two records are + * only joined if their timestamps are close to each other as defined by the given {@link + * JoinWindows}, i.e., the window defines an additional join predicate on the record + * timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} + * will be called to compute a value (with arbitrary type) for the result record. Note that the + * key is read-only and should not be modified, as this can lead to undefined behaviour. The key + * of the result record is the same as for both joining input records. If an input record key or + * value is {@code null} the record will not be included in the join operation and thus no * output record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): @@ -1315,32 +1404,35 @@ KStream join(final KStream otherStream, * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records * @param windows the specification of the {@link JoinWindows} - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key and + * within the joining window intervals * @see #leftJoin(KStream, ValueJoinerWithKey, JoinWindows) * @see #outerJoin(KStream, ValueJoinerWithKey, JoinWindows) */ - KStream join(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows); + KStream join(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows); /** - * Join records of this stream with another {@code KStream}'s records using windowed inner equi join using the - * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value - * serde}, {@link Serde the other stream's value serde}, and used state stores. - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed inner equi + * join using the {@link StreamJoined} instance for configuration of the {@link Serde key + * serde}, {@link Serde this stream's value serde}, {@link Serde the other stream's value + * serde}, and used state stores. The join is computed on the records' key with join attribute + * {@code thisKStream.key == otherKStream.key}. Furthermore, two records are only joined if + * their timestamps are close to each other as defined by the given {@link JoinWindows}, i.e., + * the window defines an additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will + * be called to compute a value (with arbitrary type) for the result record. The key of the + * result record is the same as for both joining input records. If an input record key or value + * is {@code null} the record will not be included in the join operation and thus no output + * record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -1392,35 +1484,38 @@ KStream join(final KStream otherStream, *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. * - * @param the value type of the other stream - * @param the value type of the result stream - * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param windows the specification of the {@link JoinWindows} - * @param streamJoined a {@link StreamJoined} used to configure join stores - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @param otherStream the {@code KStream} to be joined with this stream + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of + * matching records + * @param windows the specification of the {@link JoinWindows} + * @param streamJoined a {@link StreamJoined} used to configure join stores + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key and within the + * joining window intervals * @see #leftJoin(KStream, ValueJoiner, JoinWindows, StreamJoined) * @see #outerJoin(KStream, ValueJoiner, JoinWindows, StreamJoined) */ - KStream join(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final StreamJoined streamJoined); + KStream join(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows, + final StreamJoined streamJoined); /** - * Join records of this stream with another {@code KStream}'s records using windowed inner equi join using the - * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value - * serde}, {@link Serde the other stream's value serde}, and used state stores. - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} will be called to compute - * a value (with arbitrary type) for the result record. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * The key of the result record is the same as for both joining input records. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no + * Join records of this stream with another {@code KStream}'s records using windowed inner equi + * join using the {@link StreamJoined} instance for configuration of the {@link Serde key + * serde}, {@link Serde this stream's value serde}, {@link Serde the other stream's value + * serde}, and used state stores. The join is computed on the records' key with join attribute + * {@code thisKStream.key == otherKStream.key}. Furthermore, two records are only joined if + * their timestamps are close to each other as defined by the given {@link JoinWindows}, i.e., + * the window defines an additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} + * will be called to compute a value (with arbitrary type) for the result record. Note that the + * key is read-only and should not be modified, as this can lead to undefined behaviour. The key + * of the result record is the same as for both joining input records. If an input record key or + * value is {@code null} the record will not be included in the join operation and thus no * output record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): @@ -1473,37 +1568,40 @@ KStream join(final KStream otherStream, *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. * - * @param the value type of the other stream - * @param the value type of the result stream - * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param windows the specification of the {@link JoinWindows} - * @param streamJoined a {@link StreamJoined} used to configure join stores - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @param otherStream the {@code KStream} to be joined with this stream + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param windows the specification of the {@link JoinWindows} + * @param streamJoined a {@link StreamJoined} used to configure join stores + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key and + * within the joining window intervals * @see #leftJoin(KStream, ValueJoinerWithKey, JoinWindows, StreamJoined) * @see #outerJoin(KStream, ValueJoinerWithKey, JoinWindows, StreamJoined) */ - KStream join(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined); + KStream join(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined); + /** - * Join records of this stream with another {@code KStream}'s records using windowed left equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join}, all records from this stream will - * produce at least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of this {@code KStream} that does not satisfy the join predicate the provided - * {@link ValueJoiner} will be called with a {@code null} value for the other stream. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed left equi + * join with default serializers and deserializers. In contrast to {@link #join(KStream, + * ValueJoiner, JoinWindows) inner-join}, all records from this stream will produce at least one + * output record (cf. below). The join is computed on the records' key with join attribute + * {@code thisKStream.key == otherKStream.key}. Furthermore, two records are only joined if + * their timestamps are close to each other as defined by the given {@link JoinWindows}, i.e., + * the window defines an additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will + * be called to compute a value (with arbitrary type) for the result record. The key of the + * result record is the same as for both joining input records. Furthermore, for each input + * record of this {@code KStream} that does not satisfy the join predicate the provided {@link + * ValueJoiner} will be called with a {@code null} value for the other stream. If an input + * record key or value is {@code null} the record will not be included in the join operation and + * thus no output record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -1554,36 +1652,39 @@ KStream join(final KStream otherStream, * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records * @param windows the specification of the {@link JoinWindows} - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * this {@code KStream} and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key plus one for + * each non-matching record of this {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows) * @see #outerJoin(KStream, ValueJoiner, JoinWindows) */ - KStream leftJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows); + KStream leftJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows); + /** - * Join records of this stream with another {@code KStream}'s records using windowed left equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KStream, ValueJoinerWithKey, JoinWindows) inner-join}, all records from this stream will - * produce at least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} will be called to compute - * a value (with arbitrary type) for the result record. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of this {@code KStream} that does not satisfy the join predicate the provided - * {@link ValueJoinerWithKey} will be called with a {@code null} value for the other stream. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed left equi + * join with default serializers and deserializers. In contrast to {@link #join(KStream, + * ValueJoinerWithKey, JoinWindows) inner-join}, all records from this stream will produce at + * least one output record (cf. below). The join is computed on the records' key with join + * attribute {@code thisKStream.key == otherKStream.key}. Furthermore, two records are only + * joined if their timestamps are close to each other as defined by the given {@link + * JoinWindows}, i.e., the window defines an additional join predicate on the record + * timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} + * will be called to compute a value (with arbitrary type) for the result record. Note that the + * key is read-only and should not be modified, as this can lead to undefined behaviour. The key + * of the result record is the same as for both joining input records. Furthermore, for each + * input record of this {@code KStream} that does not satisfy the join predicate the provided + * {@link ValueJoinerWithKey} will be called with a {@code null} value for the other stream. If + * an input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -1634,37 +1735,39 @@ KStream leftJoin(final KStream otherStream, * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records * @param windows the specification of the {@link JoinWindows} - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one for each non-matching record of - * this {@code KStream} and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one + * for each non-matching record of this {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoinerWithKey, JoinWindows) * @see #outerJoin(KStream, ValueJoinerWithKey, JoinWindows) */ - KStream leftJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows); + KStream leftJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows); /** - * Join records of this stream with another {@code KStream}'s records using windowed left equi join using the - * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value - * serde}, {@link Serde the other stream's value serde}, and used state stores. - * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join}, all records from this stream will - * produce at least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of this {@code KStream} that does not satisfy the join predicate the provided - * {@link ValueJoiner} will be called with a {@code null} value for the other stream. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed left equi + * join using the {@link StreamJoined} instance for configuration of the {@link Serde key + * serde}, {@link Serde this stream's value serde}, {@link Serde the other stream's value + * serde}, and used state stores. In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) + * inner-join}, all records from this stream will produce at least one output record (cf. + * below). The join is computed on the records' key with join attribute {@code thisKStream.key + * == otherKStream.key}. Furthermore, two records are only joined if their timestamps are close + * to each other as defined by the given {@link JoinWindows}, i.e., the window defines an + * additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will + * be called to compute a value (with arbitrary type) for the result record. The key of the + * result record is the same as for both joining input records. Furthermore, for each input + * record of this {@code KStream} that does not satisfy the join predicate the provided {@link + * ValueJoiner} will be called with a {@code null} value for the other stream. If an input + * record key or value is {@code null} the record will not be included in the join operation and + * thus no output record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -1715,41 +1818,43 @@ KStream leftJoin(final KStream otherStream, *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. * - * @param the value type of the other stream - * @param the value type of the result stream + * @param the value type of the other stream + * @param the value type of the result stream * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of + * matching records * @param windows the specification of the {@link JoinWindows} * @param streamJoined a {@link StreamJoined} instance to configure serdes and state stores - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * this {@code KStream} and within the joining window intervals + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key plus one for + * each non-matching record of this {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows, StreamJoined) * @see #outerJoin(KStream, ValueJoiner, JoinWindows, StreamJoined) */ - KStream leftJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final StreamJoined streamJoined); + KStream leftJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows, + final StreamJoined streamJoined); /** - * Join records of this stream with another {@code KStream}'s records using windowed left equi join using the - * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value - * serde}, {@link Serde the other stream's value serde}, and used state stores. - * In contrast to {@link #join(KStream, ValueJoinerWithKey, JoinWindows) inner-join}, all records from this stream will - * produce at least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} will be called to compute - * a value (with arbitrary type) for the result record. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of this {@code KStream} that does not satisfy the join predicate the provided - * {@link ValueJoinerWithKey} will be called with a {@code null} value for the other stream. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed left equi + * join using the {@link StreamJoined} instance for configuration of the {@link Serde key + * serde}, {@link Serde this stream's value serde}, {@link Serde the other stream's value + * serde}, and used state stores. In contrast to {@link #join(KStream, ValueJoinerWithKey, + * JoinWindows) inner-join}, all records from this stream will produce at least one output + * record (cf. below). The join is computed on the records' key with join attribute {@code + * thisKStream.key == otherKStream.key}. Furthermore, two records are only joined if their + * timestamps are close to each other as defined by the given {@link JoinWindows}, i.e., the + * window defines an additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} + * will be called to compute a value (with arbitrary type) for the result record. Note that the + * key is read-only and should not be modified, as this can lead to undefined behaviour. The key + * of the result record is the same as for both joining input records. Furthermore, for each + * input record of this {@code KStream} that does not satisfy the join predicate the provided + * {@link ValueJoinerWithKey} will be called with a {@code null} value for the other stream. If + * an input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -1800,39 +1905,42 @@ KStream leftJoin(final KStream otherStream, *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. * - * @param the value type of the other stream - * @param the value type of the result stream + * @param the value type of the other stream + * @param the value type of the result stream * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records * @param windows the specification of the {@link JoinWindows} * @param streamJoined a {@link StreamJoined} instance to configure serdes and state stores - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one for each non-matching record of - * this {@code KStream} and within the joining window intervals + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one + * for each non-matching record of this {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoinerWithKey, JoinWindows, StreamJoined) * @see #outerJoin(KStream, ValueJoinerWithKey, JoinWindows, StreamJoined) */ - KStream leftJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined); + KStream leftJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined); + /** - * Join records of this stream with another {@code KStream}'s records using windowed outer equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join} or - * {@link #leftJoin(KStream, ValueJoiner, JoinWindows) left-join}, all records from both streams will produce at - * least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of both {@code KStream}s that does not satisfy the join predicate the provided - * {@link ValueJoiner} will be called with a {@code null} value for the this/other stream, respectively. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed outer equi + * join with default serializers and deserializers. In contrast to {@link #join(KStream, + * ValueJoiner, JoinWindows) inner-join} or {@link #leftJoin(KStream, ValueJoiner, JoinWindows) + * left-join}, all records from both streams will produce at least one output record (cf. + * below). The join is computed on the records' key with join attribute {@code thisKStream.key + * == otherKStream.key}. Furthermore, two records are only joined if their timestamps are close + * to each other as defined by the given {@link JoinWindows}, i.e., the window defines an + * additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will + * be called to compute a value (with arbitrary type) for the result record. The key of the + * result record is the same as for both joining input records. Furthermore, for each input + * record of both {@code KStream}s that does not satisfy the join predicate the provided {@link + * ValueJoiner} will be called with a {@code null} value for the this/other stream, + * respectively. If an input record key or value is {@code null} the record will not be included + * in the join operation and thus no output record will be added to the resulting {@code + * KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -1883,37 +1991,40 @@ KStream leftJoin(final KStream otherStream, * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records * @param windows the specification of the {@link JoinWindows} - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * both {@code KStream} and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key plus one for + * each non-matching record of both {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows) * @see #leftJoin(KStream, ValueJoiner, JoinWindows) */ - KStream outerJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows); + KStream outerJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows); + /** - * Join records of this stream with another {@code KStream}'s records using windowed outer equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KStream, ValueJoinerWithKey, JoinWindows) inner-join} or - * {@link #leftJoin(KStream, ValueJoinerWithKey, JoinWindows) left-join}, all records from both streams will produce at - * least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} will be called to compute - * a value (with arbitrary type) for the result record. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of both {@code KStream}s that does not satisfy the join predicate the provided - * {@link ValueJoinerWithKey} will be called with a {@code null} value for the this/other stream, respectively. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed outer equi + * join with default serializers and deserializers. In contrast to {@link #join(KStream, + * ValueJoinerWithKey, JoinWindows) inner-join} or {@link #leftJoin(KStream, ValueJoinerWithKey, + * JoinWindows) left-join}, all records from both streams will produce at least one output + * record (cf. below). The join is computed on the records' key with join attribute {@code + * thisKStream.key == otherKStream.key}. Furthermore, two records are only joined if their + * timestamps are close to each other as defined by the given {@link JoinWindows}, i.e., the + * window defines an additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} + * will be called to compute a value (with arbitrary type) for the result record. Note that the + * key is read-only and should not be modified, as this can lead to undefined behaviour. The key + * of the result record is the same as for both joining input records. Furthermore, for each + * input record of both {@code KStream}s that does not satisfy the join predicate the provided + * {@link ValueJoinerWithKey} will be called with a {@code null} value for the this/other + * stream, respectively. If an input record key or value is {@code null} the record will not be + * included in the join operation and thus no output record will be added to the resulting + * {@code KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -1964,38 +2075,40 @@ KStream outerJoin(final KStream otherStream, * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records * @param windows the specification of the {@link JoinWindows} - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one for each non-matching record of - * both {@code KStream} and within the joining window intervals + * @param the value type of the other stream + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one + * for each non-matching record of both {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoinerWithKey, JoinWindows) * @see #leftJoin(KStream, ValueJoinerWithKey, JoinWindows) */ - KStream outerJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows); + KStream outerJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows); /** - * Join records of this stream with another {@code KStream}'s records using windowed outer equi join using the - * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value - * serde}, {@link Serde the other stream's value serde}, and used state stores. - * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join} or - * {@link #leftJoin(KStream, ValueJoiner, JoinWindows) left-join}, all records from both streams will produce at - * least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of both {@code KStream}s that does not satisfy the join predicate the provided - * {@link ValueJoiner} will be called with a {@code null} value for this/other stream, respectively. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed outer equi + * join using the {@link StreamJoined} instance for configuration of the {@link Serde key + * serde}, {@link Serde this stream's value serde}, {@link Serde the other stream's value + * serde}, and used state stores. In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) + * inner-join} or {@link #leftJoin(KStream, ValueJoiner, JoinWindows) left-join}, all records + * from both streams will produce at least one output record (cf. below). The join is computed + * on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. + * Furthermore, two records are only joined if their timestamps are close to each other as + * defined by the given {@link JoinWindows}, i.e., the window defines an additional join + * predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will + * be called to compute a value (with arbitrary type) for the result record. The key of the + * result record is the same as for both joining input records. Furthermore, for each input + * record of both {@code KStream}s that does not satisfy the join predicate the provided {@link + * ValueJoiner} will be called with a {@code null} value for this/other stream, respectively. If + * an input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -2046,42 +2159,45 @@ KStream outerJoin(final KStream otherStream, *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. * - * @param the value type of the other stream - * @param the value type of the result stream + * @param the value type of the other stream + * @param the value type of the result stream * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of + * matching records * @param windows the specification of the {@link JoinWindows} * @param streamJoined a {@link StreamJoined} instance to configure serdes and state stores - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * both {@code KStream} and within the joining window intervals + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key plus one for + * each non-matching record of both {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows, StreamJoined) * @see #leftJoin(KStream, ValueJoiner, JoinWindows, StreamJoined) */ - KStream outerJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final StreamJoined streamJoined); + KStream outerJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows, + final StreamJoined streamJoined); /** - * Join records of this stream with another {@code KStream}'s records using windowed outer equi join using the - * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value - * serde}, {@link Serde the other stream's value serde}, and used state stores. - * In contrast to {@link #join(KStream, ValueJoinerWithKey, JoinWindows) inner-join} or - * {@link #leftJoin(KStream, ValueJoinerWithKey, JoinWindows) left-join}, all records from both streams will produce at - * least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

- * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} will be called to compute - * a value (with arbitrary type) for the result record. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of both {@code KStream}s that does not satisfy the join predicate the provided - * {@link ValueJoinerWithKey} will be called with a {@code null} value for this/other stream, respectively. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. + * Join records of this stream with another {@code KStream}'s records using windowed outer equi + * join using the {@link StreamJoined} instance for configuration of the {@link Serde key + * serde}, {@link Serde this stream's value serde}, {@link Serde the other stream's value + * serde}, and used state stores. In contrast to {@link #join(KStream, ValueJoinerWithKey, + * JoinWindows) inner-join} or {@link #leftJoin(KStream, ValueJoinerWithKey, JoinWindows) + * left-join}, all records from both streams will produce at least one output record (cf. + * below). The join is computed on the records' key with join attribute {@code thisKStream.key + * == otherKStream.key}. Furthermore, two records are only joined if their timestamps are close + * to each other as defined by the given {@link JoinWindows}, i.e., the window defines an + * additional join predicate on the record timestamps. + *

+ * For each pair of records meeting both join predicates the provided {@link ValueJoinerWithKey} + * will be called to compute a value (with arbitrary type) for the result record. Note that the + * key is read-only and should not be modified, as this can lead to undefined behaviour. The key + * of the result record is the same as for both joining input records. Furthermore, for each + * input record of both {@code KStream}s that does not satisfy the join predicate the provided + * {@link ValueJoinerWithKey} will be called with a {@code null} value for this/other stream, + * respectively. If an input record key or value is {@code null} the record will not be included + * in the join operation and thus no output record will be added to the resulting {@code + * KStream}. *

* Example (assuming all input records belong to the correct windows): *

@@ -2132,37 +2248,39 @@ KStream outerJoin(final KStream otherStream, *

* You can retrieve all generated internal topic names via {@link Topology#describe()}. * - * @param the value type of the other stream - * @param the value type of the result stream + * @param the value type of the other stream + * @param the value type of the result stream * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records * @param windows the specification of the {@link JoinWindows} * @param streamJoined a {@link StreamJoined} instance to configure serdes and state stores - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one for each non-matching record of - * both {@code KStream} and within the joining window intervals + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key plus one + * for each non-matching record of both {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoinerWithKey, JoinWindows, StreamJoined) * @see #leftJoin(KStream, ValueJoinerWithKey, JoinWindows, StreamJoined) */ - KStream outerJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined); + KStream outerJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined); + /** - * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join with default - * serializers and deserializers. - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link KTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join + * with default serializers and deserializers. The join is a primary key table lookup join with + * join attribute {@code stream.key == table.key}. "Table lookup join" means, that results are + * only computed if {@code KStream} records are processed. This is done by performing a lookup + * for matching records in the current (i.e., processing time) internal {@link KTable} + * state. In contrast, processing {@link KTable} input records will only update the internal + * {@link KTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link KTable} the + * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the + * result record. The key of the result record is the same as for both joining input records. If + * an {@code KStream} input record key or value is {@code null} the record will not be included + * in the join operation and thus no output record will be added to the resulting {@code + * KStream}. *

* Example: *

@@ -2213,34 +2331,35 @@ KStream outerJoin(final KStream otherStream, * correctly on its key. * * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key * @see #leftJoin(KTable, ValueJoiner) * @see #join(GlobalKTable, KeyValueMapper, ValueJoiner) */ - KStream join(final KTable table, - final ValueJoiner joiner); + KStream join(final KTable table, + final ValueJoiner joiner); /** - * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join with default - * serializers and deserializers. - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link KTable} the provided - * {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * - * The key of the result record is the same as for both joining input records. - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join + * with default serializers and deserializers. The join is a primary key table lookup join with + * join attribute {@code stream.key == table.key}. "Table lookup join" means, that results are + * only computed if {@code KStream} records are processed. This is done by performing a lookup + * for matching records in the current (i.e., processing time) internal {@link KTable} + * state. In contrast, processing {@link KTable} input records will only update the internal + * {@link KTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link KTable} the + * provided {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) + * for the result record. Note that the key is read-only and should not be modified, as this can + * lead to undefined behaviour. + *

+ * The key of the result record is the same as for both joining input records. If an {@code + * KStream} input record key or value is {@code null} the record will not be included in the + * join operation and thus no output record will be added to the resulting {@code KStream}. *

* Example: *

@@ -2291,32 +2410,33 @@ KStream join(final KTable table, * correctly on its key. * * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key * @see #leftJoin(KTable, ValueJoinerWithKey) * @see #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream join(final KTable table, - final ValueJoinerWithKey joiner); + KStream join(final KTable table, + final ValueJoinerWithKey joiner); /** - * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join with default - * serializers and deserializers. - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link KTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join + * with default serializers and deserializers. The join is a primary key table lookup join with + * join attribute {@code stream.key == table.key}. "Table lookup join" means, that results are + * only computed if {@code KStream} records are processed. This is done by performing a lookup + * for matching records in the current (i.e., processing time) internal {@link KTable} + * state. In contrast, processing {@link KTable} input records will only update the internal + * {@link KTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link KTable} the + * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the + * result record. The key of the result record is the same as for both joining input records. If + * an {@code KStream} input record key or value is {@code null} the record will not be included + * in the join operation and thus no output record will be added to the resulting {@code + * KStream}. *

* Example: *

@@ -2367,37 +2487,39 @@ KStream join(final KTable table, * correctly on its key. * * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param joined a {@link Joined} instance that defines the serdes to - * be used to serialize/deserialize inputs of the joined streams - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param joined a {@link Joined} instance that defines the serdes to be used to + * serialize/deserialize inputs of the joined streams + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one for each matched record-pair with the same key * @see #leftJoin(KTable, ValueJoiner, Joined) * @see #join(GlobalKTable, KeyValueMapper, ValueJoiner) */ - KStream join(final KTable table, - final ValueJoiner joiner, - final Joined joined); + KStream join(final KTable table, + final ValueJoiner joiner, + final Joined joined); /** - * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join with default - * serializers and deserializers. - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link KTable} the provided - * {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join + * with default serializers and deserializers. The join is a primary key table lookup join with + * join attribute {@code stream.key == table.key}. "Table lookup join" means, that results are + * only computed if {@code KStream} records are processed. This is done by performing a lookup + * for matching records in the current (i.e., processing time) internal {@link KTable} + * state. In contrast, processing {@link KTable} input records will only update the internal + * {@link KTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link KTable} the + * provided {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) + * for the result record. The key of the result record is the same as for both joining input + * records. Note that the key is read-only and should not be modified, as this can lead to + * undefined behaviour. + *

+ * If an {@code KStream} input record key or value is {@code null} the record will not be + * included in the join operation and thus no output record will be added to the resulting + * {@code KStream}. *

* Example: *

@@ -2448,38 +2570,39 @@ KStream join(final KTable table, * correctly on its key. * * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param joined a {@link Joined} instance that defines the serdes to - * be used to serialize/deserialize inputs of the joined streams - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one for each matched record-pair with the same key + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param joined a {@link Joined} instance that defines the serdes to be used to + * serialize/deserialize inputs of the joined streams + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one for each matched record-pair with the same key * @see #leftJoin(KTable, ValueJoinerWithKey, Joined) * @see #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream join(final KTable table, - final ValueJoinerWithKey joiner, - final Joined joined); + KStream join(final KTable table, + final ValueJoinerWithKey joiner, + final Joined joined); /** - * Join records of this stream with {@link KTable}'s records using non-windowed left equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from this stream will produce an - * output record (cf. below). - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link KTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoiner}. - * The key of the result record is the same as for both joining input records. - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed left equi join + * with default serializers and deserializers. In contrast to {@link #join(KTable, ValueJoiner) + * inner-join}, all records from this stream will produce an output record (cf. below). The join + * is a primary key table lookup join with join attribute {@code stream.key == table.key}. + * "Table lookup join" means, that results are only computed if {@code KStream} records are + * processed. This is done by performing a lookup for matching records in the current + * (i.e., processing time) internal {@link KTable} state. In contrast, processing {@link KTable} + * input records will only update the internal {@link KTable} state and will not produce any + * result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * KTable} the provided {@link ValueJoiner} will be called to compute a value (with arbitrary + * type) for the result record. If no {@link KTable} record was found during lookup, a {@code + * null} value will be provided to {@link ValueJoiner}. The key of the result record is the same + * as for both joining input records. If an {@code KStream} input record key or value is {@code + * null} the record will not be included in the join operation and thus no output record will be + * added to the resulting {@code KStream}. *

* Example: *

@@ -2530,36 +2653,37 @@ KStream join(final KTable table, * correctly on its key. * * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@code KStream} record + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one output for each input {@code KStream} record * @see #join(KTable, ValueJoiner) * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) */ - KStream leftJoin(final KTable table, - final ValueJoiner joiner); + KStream leftJoin(final KTable table, + final ValueJoiner joiner); /** - * Join records of this stream with {@link KTable}'s records using non-windowed left equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KTable, ValueJoinerWithKey) inner-join}, all records from this stream will produce an - * output record (cf. below). - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link KTable} the provided - * {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoinerWithKey}. - * The key of the result record is the same as for both joining input records. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed left equi join + * with default serializers and deserializers. In contrast to {@link #join(KTable, + * ValueJoinerWithKey) inner-join}, all records from this stream will produce an output record + * (cf. below). The join is a primary key table lookup join with join attribute {@code + * stream.key == table.key}. "Table lookup join" means, that results are only computed if {@code + * KStream} records are processed. This is done by performing a lookup for matching records in + * the current (i.e., processing time) internal {@link KTable} state. In contrast, + * processing {@link KTable} input records will only update the internal {@link KTable} state + * and will not produce any result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * KTable} the provided {@link ValueJoinerWithKey} will be called to compute a value (with + * arbitrary type) for the result record. If no {@link KTable} record was found during lookup, a + * {@code null} value will be provided to {@link ValueJoinerWithKey}. The key of the result + * record is the same as for both joining input records. Note that the key is read-only and + * should not be modified, as this can lead to undefined behaviour. If an {@code KStream} input + * record key or value is {@code null} the record will not be included in the join operation and + * thus no output record will be added to the resulting {@code KStream}. *

* Example: *

@@ -2610,35 +2734,36 @@ KStream leftJoin(final KTable table, * correctly on its key. * * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one output for each input {@code KStream} record + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one output for each input {@code KStream} record * @see #join(KTable, ValueJoinerWithKey) * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream leftJoin(final KTable table, - final ValueJoinerWithKey joiner); + KStream leftJoin(final KTable table, + final ValueJoinerWithKey joiner); /** - * Join records of this stream with {@link KTable}'s records using non-windowed left equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from this stream will produce an - * output record (cf. below). - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link KTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoiner}. - * The key of the result record is the same as for both joining input records. - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed left equi join + * with default serializers and deserializers. In contrast to {@link #join(KTable, ValueJoiner) + * inner-join}, all records from this stream will produce an output record (cf. below). The join + * is a primary key table lookup join with join attribute {@code stream.key == table.key}. + * "Table lookup join" means, that results are only computed if {@code KStream} records are + * processed. This is done by performing a lookup for matching records in the current + * (i.e., processing time) internal {@link KTable} state. In contrast, processing {@link KTable} + * input records will only update the internal {@link KTable} state and will not produce any + * result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * KTable} the provided {@link ValueJoiner} will be called to compute a value (with arbitrary + * type) for the result record. If no {@link KTable} record was found during lookup, a {@code + * null} value will be provided to {@link ValueJoiner}. The key of the result record is the same + * as for both joining input records. If an {@code KStream} input record key or value is {@code + * null} the record will not be included in the join operation and thus no output record will be + * added to the resulting {@code KStream}. *

* Example: *

@@ -2688,40 +2813,41 @@ KStream leftJoin(final KTable table, * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned * correctly on its key. * - * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param joined a {@link Joined} instance that defines the serdes to - * be used to serialize/deserialize inputs and outputs of the joined streams - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@code KStream} record + * @param table the {@link KTable} to be joined with this stream + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param joined a {@link Joined} instance that defines the serdes to be used to + * serialize/deserialize inputs and outputs of the joined streams + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one output for each input {@code KStream} record * @see #join(KTable, ValueJoiner, Joined) * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) */ - KStream leftJoin(final KTable table, - final ValueJoiner joiner, - final Joined joined); + KStream leftJoin(final KTable table, + final ValueJoiner joiner, + final Joined joined); /** - * Join records of this stream with {@link KTable}'s records using non-windowed left equi join with default - * serializers and deserializers. - * In contrast to {@link #join(KTable, ValueJoinerWithKey) inner-join}, all records from this stream will produce an - * output record (cf. below). - * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current (i.e., processing time) internal - * {@link KTable} state. - * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and - * will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link KTable} the provided - * {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoinerWithKey}. - * The key of the result record is the same as for both joining input records. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. + * Join records of this stream with {@link KTable}'s records using non-windowed left equi join + * with default serializers and deserializers. In contrast to {@link #join(KTable, + * ValueJoinerWithKey) inner-join}, all records from this stream will produce an output record + * (cf. below). The join is a primary key table lookup join with join attribute {@code + * stream.key == table.key}. "Table lookup join" means, that results are only computed if {@code + * KStream} records are processed. This is done by performing a lookup for matching records in + * the current (i.e., processing time) internal {@link KTable} state. In contrast, + * processing {@link KTable} input records will only update the internal {@link KTable} state + * and will not produce any result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * KTable} the provided {@link ValueJoinerWithKey} will be called to compute a value (with + * arbitrary type) for the result record. If no {@link KTable} record was found during lookup, a + * {@code null} value will be provided to {@link ValueJoinerWithKey}. The key of the result + * record is the same as for both joining input records. Note that the key is read-only and + * should not be modified, as this can lead to undefined behaviour. If an {@code KStream} input + * record key or value is {@code null} the record will not be included in the join operation and + * thus no output record will be added to the resulting {@code KStream}. *

* Example: *

@@ -2771,48 +2897,49 @@ KStream leftJoin(final KTable table, * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned * correctly on its key. * - * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param joined a {@link Joined} instance that defines the serdes to - * be used to serialize/deserialize inputs and outputs of the joined streams - * @param the value type of the table - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one output for each input {@code KStream} record + * @param table the {@link KTable} to be joined with this stream + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param joined a {@link Joined} instance that defines the serdes to be used to + * serialize/deserialize inputs and outputs of the joined streams + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one output for each input {@code KStream} record * @see #join(KTable, ValueJoinerWithKey, Joined) * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream leftJoin(final KTable table, - final ValueJoinerWithKey joiner, - final Joined joined); + KStream leftJoin(final KTable table, + final ValueJoinerWithKey joiner, + final Joined joined); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi join. - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as the key of this {@code KStream}. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, no output record will be added to the - * resulting {@code KStream}. - * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@code KStream} record + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi + * join. The join is a primary key table lookup join with join attribute {@code + * keyValueMapper.map(stream.keyValue) == table.key}. "Table lookup join" means, that results + * are only computed if {@code KStream} records are processed. This is done by performing a + * lookup for matching records in the current internal {@link GlobalKTable} state. In + * contrast, processing {@link GlobalKTable} input records will only update the internal {@link + * GlobalKTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the + * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the + * result record. The key of the result record is the same as the key of this {@code KStream}. + * If a {@code KStream} input record key or value is {@code null} the record will not be + * included in the join operation and thus no output record will be added to the resulting + * {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no match exists, no + * output record will be added to the resulting {@code KStream}. + * + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one output for each input {@code KStream} record * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) */ KStream join(final GlobalKTable globalTable, @@ -2820,282 +2947,284 @@ KStream join(final GlobalKTable globalTable, final ValueJoiner joiner); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi join. - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the provided - * {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as the key of this {@code KStream}. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, no output record will be added to the - * resulting {@code KStream}. - * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one output for each input {@code KStream} record + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi + * join. The join is a primary key table lookup join with join attribute {@code + * keyValueMapper.map(stream.keyValue) == table.key}. "Table lookup join" means, that results + * are only computed if {@code KStream} records are processed. This is done by performing a + * lookup for matching records in the current internal {@link GlobalKTable} state. In + * contrast, processing {@link GlobalKTable} input records will only update the internal {@link + * GlobalKTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the + * provided {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) + * for the result record. The key of the result record is the same as the key of this {@code + * KStream}. Note that the key is read-only and should not be modified, as this can lead to + * undefined behaviour. If a {@code KStream} input record key or value is {@code null} the + * record will not be included in the join operation and thus no output record will be added to + * the resulting {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no + * match exists, no output record will be added to the resulting {@code KStream}. + * + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one output for each input {@code KStream} record * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream join(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey joiner); + KStream join(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey joiner); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi join. - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as the key of this {@code KStream}. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, no output record will be added to the - * resulting {@code KStream}. - * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param named a {@link Named} config used to name the processor in the topology - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@code KStream} record + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi + * join. The join is a primary key table lookup join with join attribute {@code + * keyValueMapper.map(stream.keyValue) == table.key}. "Table lookup join" means, that results + * are only computed if {@code KStream} records are processed. This is done by performing a + * lookup for matching records in the current internal {@link GlobalKTable} state. In + * contrast, processing {@link GlobalKTable} input records will only update the internal {@link + * GlobalKTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the + * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the + * result record. The key of the result record is the same as the key of this {@code KStream}. + * If a {@code KStream} input record key or value is {@code null} the record will not be + * included in the join operation and thus no output record will be added to the resulting + * {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no match exists, no + * output record will be added to the resulting {@code KStream}. + * + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param named a {@link Named} config used to name the processor in the topology + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one output for each input {@code KStream} record * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) */ - KStream join(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoiner joiner, - final Named named); + KStream join(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoiner joiner, + final Named named); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi join. - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the provided - * {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as the key of this {@code KStream}. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, no output record will be added to the - * resulting {@code KStream}. - * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param named a {@link Named} config used to name the processor in the topology - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one output for each input {@code KStream} record + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi + * join. The join is a primary key table lookup join with join attribute {@code + * keyValueMapper.map(stream.keyValue) == table.key}. "Table lookup join" means, that results + * are only computed if {@code KStream} records are processed. This is done by performing a + * lookup for matching records in the current internal {@link GlobalKTable} state. In + * contrast, processing {@link GlobalKTable} input records will only update the internal {@link + * GlobalKTable} state and will not produce any result records. + *

+ * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the + * provided {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) + * for the result record. The key of the result record is the same as the key of this {@code + * KStream}. Note that the key is read-only and should not be modified, as this can lead to + * undefined behaviour. If a {@code KStream} input record key or value is {@code null} the + * record will not be included in the join operation and thus no output record will be added to + * the resulting {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no + * match exists, no output record will be added to the resulting {@code KStream}. + * + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param joiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param named a {@link Named} config used to name the processor in the topology + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one output for each input {@code KStream} record * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream join(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey joiner, - final Named named); + KStream join(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey joiner, + final Named named); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi join. - * In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner) inner-join}, all records from this stream - * will produce an output record (cf. below). - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link GlobalKTable} the - * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as this {@code KStream}. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, a {@code null} value will be - * provided to {@link ValueJoiner}. - * If no {@link GlobalKTable} record was found during lookup, a {@code null} value will be provided to - * {@link ValueJoiner}. - * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param valueJoiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@code KStream} record + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi + * join. In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner) inner-join}, all + * records from this stream will produce an output record (cf. below). The join is a primary key + * table lookup join with join attribute {@code keyValueMapper.map(stream.keyValue) == + * table.key}. "Table lookup join" means, that results are only computed if {@code KStream} + * records are processed. This is done by performing a lookup for matching records in the + * current internal {@link GlobalKTable} state. In contrast, processing {@link + * GlobalKTable} input records will only update the internal {@link GlobalKTable} state and will + * not produce any result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * GlobalKTable} the provided {@link ValueJoiner} will be called to compute a value (with + * arbitrary type) for the result record. The key of the result record is the same as this + * {@code KStream}. If a {@code KStream} input record key or value is {@code null} the record + * will not be included in the join operation and thus no output record will be added to the + * resulting {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no match + * exists, a {@code null} value will be provided to {@link ValueJoiner}. If no {@link + * GlobalKTable} record was found during lookup, a {@code null} value will be provided to {@link + * ValueJoiner}. + * + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param valueJoiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one output for each input {@code KStream} record * @see #join(GlobalKTable, KeyValueMapper, ValueJoiner) */ - KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoiner valueJoiner); + KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoiner valueJoiner); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi join. - * In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) inner-join}, all records from this stream - * will produce an output record (cf. below). - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link GlobalKTable} the - * provided {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as this {@code KStream}. - * Note that the key is read-only and should not be modified, as this can lead to undefined behaviour. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, a {@code null} value will be - * provided to {@link ValueJoinerWithKey}. - * If no {@link GlobalKTable} record was found during lookup, a {@code null} value will be provided to + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi + * join. In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) + * inner-join}, all records from this stream will produce an output record (cf. below). The join + * is a primary key table lookup join with join attribute {@code keyValueMapper.map(stream.keyValue) + * == table.key}. "Table lookup join" means, that results are only computed if {@code KStream} + * records are processed. This is done by performing a lookup for matching records in the + * current internal {@link GlobalKTable} state. In contrast, processing {@link + * GlobalKTable} input records will only update the internal {@link GlobalKTable} state and will + * not produce any result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * GlobalKTable} the provided {@link ValueJoinerWithKey} will be called to compute a value (with + * arbitrary type) for the result record. The key of the result record is the same as this + * {@code KStream}. Note that the key is read-only and should not be modified, as this can lead + * to undefined behaviour. If a {@code KStream} input record key or value is {@code null} the + * record will not be included in the join operation and thus no output record will be added to + * the resulting {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no + * match exists, a {@code null} value will be provided to {@link ValueJoinerWithKey}. If no + * {@link GlobalKTable} record was found during lookup, a {@code null} value will be provided to * {@link ValueJoiner}. * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param valueJoiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one output for each input {@code KStream} record + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param valueJoiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one output for each input {@code KStream} record * @see #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey valueJoiner); + KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey valueJoiner); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi join. - * In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner) inner-join}, all records from this stream - * will produce an output record (cf. below). - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link GlobalKTable} the - * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as this {@code KStream}. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, a {@code null} value will be - * provided to {@link ValueJoiner}. - * If no {@link GlobalKTable} record was found during lookup, a {@code null} value will be provided to - * {@link ValueJoiner}. - * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param valueJoiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param named a {@link Named} config used to name the processor in the topology - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@code KStream} record + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi + * join. In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner) inner-join}, all + * records from this stream will produce an output record (cf. below). The join is a primary key + * table lookup join with join attribute {@code keyValueMapper.map(stream.keyValue) == + * table.key}. "Table lookup join" means, that results are only computed if {@code KStream} + * records are processed. This is done by performing a lookup for matching records in the + * current internal {@link GlobalKTable} state. In contrast, processing {@link + * GlobalKTable} input records will only update the internal {@link GlobalKTable} state and will + * not produce any result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * GlobalKTable} the provided {@link ValueJoiner} will be called to compute a value (with + * arbitrary type) for the result record. The key of the result record is the same as this + * {@code KStream}. If a {@code KStream} input record key or value is {@code null} the record + * will not be included in the join operation and thus no output record will be added to the + * resulting {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no match + * exists, a {@code null} value will be provided to {@link ValueJoiner}. If no {@link + * GlobalKTable} record was found during lookup, a {@code null} value will be provided to {@link + * ValueJoiner}. + * + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param valueJoiner a {@link ValueJoiner} that computes the join result for a pair of matching + * records + * @param named a {@link Named} config used to name the processor in the topology + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoiner}, one output for each input {@code KStream} record * @see #join(GlobalKTable, KeyValueMapper, ValueJoiner) */ - KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoiner valueJoiner, - final Named named); + KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoiner valueJoiner, + final Named named); /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi join. - * In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) inner-join}, all records from this stream - * will produce an output record (cf. below). - * The join is a primary key table lookup join with join attribute - * {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} - * state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} - * state and will not produce any result records. - *

- * For each {@code KStream} record whether or not it finds a corresponding record in {@link GlobalKTable} the - * provided {@link ValueJoinerWithKey} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as this {@code KStream}. - * If a {@code KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@code KStream}. - * If {@code keyValueMapper} returns {@code null} implying no match exists, a {@code null} value will be - * provided to {@link ValueJoinerWithKey}. - * If no {@link GlobalKTable} record was found during lookup, a {@code null} value will be provided to - * {@link ValueJoinerWithKey}. - * - * @param globalTable the {@link GlobalKTable} to be joined with this stream - * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param valueJoiner a {@link ValueJoinerWithKey} that computes the join result for a pair of matching records - * @param named a {@link Named} config used to name the processor in the topology - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@code KStream} - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoinerWithKey}, one output for each input {@code KStream} record + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi + * join. In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) + * inner-join}, all records from this stream will produce an output record (cf. below). The join + * is a primary key table lookup join with join attribute {@code keyValueMapper.map(stream.keyValue) + * == table.key}. "Table lookup join" means, that results are only computed if {@code KStream} + * records are processed. This is done by performing a lookup for matching records in the + * current internal {@link GlobalKTable} state. In contrast, processing {@link + * GlobalKTable} input records will only update the internal {@link GlobalKTable} state and will + * not produce any result records. + *

+ * For each {@code KStream} record whether or not it finds a corresponding record in {@link + * GlobalKTable} the provided {@link ValueJoinerWithKey} will be called to compute a value (with + * arbitrary type) for the result record. The key of the result record is the same as this + * {@code KStream}. If a {@code KStream} input record key or value is {@code null} the record + * will not be included in the join operation and thus no output record will be added to the + * resulting {@code KStream}. If {@code keyValueMapper} returns {@code null} implying no match + * exists, a {@code null} value will be provided to {@link ValueJoinerWithKey}. If no {@link + * GlobalKTable} record was found during lookup, a {@code null} value will be provided to {@link + * ValueJoinerWithKey}. + * + * @param globalTable the {@link GlobalKTable} to be joined with this stream + * @param keySelector instance of {@link KeyValueMapper} used to map from the (key, value) of + * this stream to the key of the {@link GlobalKTable} + * @param valueJoiner a {@link ValueJoinerWithKey} that computes the join result for a pair of + * matching records + * @param named a {@link Named} config used to name the processor in the topology + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the + * given {@link ValueJoinerWithKey}, one output for each input {@code KStream} record * @see #join(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) */ - KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey valueJoiner, - final Named named); + KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey valueJoiner, + final Named named); /** - * Transform each record of the input stream into zero or one record in the output stream (both key and value type - * can be altered arbitrarily). - * A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and - * returns zero or one output record. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #map(KeyValueMapper) map()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #map(KeyValueMapper) map()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}, - * the processing progress can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform each record of the input stream into zero or one record in the output stream (both + * key and value type can be altered arbitrarily). A {@link Transformer} (provided by the given + * {@link TransformerSupplier}) is applied to each input record and returns zero or one output + * record. Thus, an input record {@code } can be transformed into an output record {@code + * }. Attaching a state store makes this a stateful record-by-record operation (cf. + * {@link #map(KeyValueMapper) map()}). If you choose not to attach one, this operation is + * similar to the stateless {@link #map(KeyValueMapper) map()} but allows access to the {@code + * ProcessorContext} and record metadata. This is essentially mixing the Processor API into the + * DSL, and provides all the functionality of the PAPI. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}, the + * processing progress can be observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3111,8 +3240,9 @@  KStream leftJoin(final GlobalKTable globalTable,
      *     }
      * }, "myTransformState");
      * }
- * The second strategy is for the given {@link TransformerSupplier} to implement {@link ConnectedStoreProvider#stores()}, - * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer. + * The second strategy is for the given {@link TransformerSupplier} to implement {@link + * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be + * automatically added to the topology and connected to the transformer. *
{@code
      * class MyTransformerSupplier implements TransformerSupplier {
      *     // supply transformer
@@ -3136,12 +3266,11 @@  KStream leftJoin(final GlobalKTable globalTable,
      * KStream outputStream = inputStream.transform(new MyTransformerSupplier());
      * }
*

- * With either strategy, within the {@link Transformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link Transformer} must return a {@link KeyValue} type in {@link Transformer#transform(Object, Object) - * transform()}. - * The return value of {@link Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, + * With either strategy, within the {@link Transformer}, the state is obtained via the {@link + * ProcessorContext}. To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * punctuate()}, a schedule must be registered. The {@link Transformer} must return a {@link + * KeyValue} type in {@link Transformer#transform(Object, Object) transform()}. The return value + * of {@link Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, * in which case no record is emitted. *

{@code
      * class MyTransformer implements Transformer {
@@ -3165,64 +3294,70 @@  KStream leftJoin(final GlobalKTable globalTable,
      *     }
      * }
      * }
- * Even if any upstream operation was key-changing, no auto-repartition is triggered. - * If repartitioning is required, a call to {@link #repartition()} should be performed before {@code transform()}. - *

- * Transforming records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@code KStream}. - * (cf. {@link #transformValues(ValueTransformerSupplier, String...) transformValues()} ) - *

- * Note that it is possible to emit multiple records for each input record by using - * {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} in - * {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be - * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should - * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * If in {@link Transformer#transform(Object, Object) Transformer#transform()} multiple records need to be emitted - * for each input record, it is recommended to use {@link #flatTransform(TransformerSupplier, String...) - * flatTransform()}. - * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating - * a single {@link Transformer} object and returning the same object reference in {@link TransformerSupplier#get()} would be - * a violation of the supplier pattern and leads to runtime exceptions. - * - * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly constructed - * {@link Transformer} - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the key type of the new stream - * @param the value type of the new stream - * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) + * Even if any upstream operation was key-changing, no auto-repartition is triggered. If + * repartitioning is required, a call to {@link #repartition()} should be performed before + * {@code transform()}. + *

+ * Transforming records might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #transformValues(ValueTransformerSupplier, String...) transformValues()} ) + *

+ * Note that it is possible to emit multiple records for each input record by using {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} in + * {@link Transformer#transform(Object, Object) Transformer#transform()} and {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. Be + * aware that a mismatch between the types of the emitted records and the type of the stream + * would only be detected at runtime. To ensure type-safety at compile-time, {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} + * should not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} + * and {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()}. If in {@link Transformer#transform(Object, Object) + * Transformer#transform()} multiple records need to be emitted for each input record, it is + * recommended to use {@link #flatTransform(TransformerSupplier, String...) flatTransform()}. + * The supplier should always generate a new instance each time {@link + * TransformerSupplier#get()} gets called. Creating a single {@link Transformer} object and + * returning the same object reference in {@link TransformerSupplier#get()} would be a violation + * of the supplier pattern and leads to runtime exceptions. + * + * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly + * constructed {@link Transformer} + * @param stateStoreNames the names of the state stores used by the processor; not required + * if the supplier implements {@link ConnectedStoreProvider#stores()} + * @param the key type of the new stream + * @param the value type of the new stream + * @return a {@code KStream} that contains more or less records with new key and value (possibly + * of different type) * @see #map(KeyValueMapper) * @see #flatTransform(TransformerSupplier, String...) * @see #transformValues(ValueTransformerSupplier, String...) * @see #transformValues(ValueTransformerWithKeySupplier, String...) * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ - KStream transform(final TransformerSupplier> transformerSupplier, - final String... stateStoreNames); + KStream transform( + final TransformerSupplier> transformerSupplier, + final String... stateStoreNames); /** - * Transform each record of the input stream into zero or one record in the output stream (both key and value type - * can be altered arbitrarily). - * A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and - * returns zero or one output record. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #map(KeyValueMapper) map()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #map(KeyValueMapper) map()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}, - * the processing progress can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform each record of the input stream into zero or one record in the output stream (both + * key and value type can be altered arbitrarily). A {@link Transformer} (provided by the given + * {@link TransformerSupplier}) is applied to each input record and returns zero or one output + * record. Thus, an input record {@code } can be transformed into an output record {@code + * }. Attaching a state store makes this a stateful record-by-record operation (cf. + * {@link #map(KeyValueMapper) map()}). If you choose not to attach one, this operation is + * similar to the stateless {@link #map(KeyValueMapper) map()} but allows access to the {@code + * ProcessorContext} and record metadata. This is essentially mixing the Processor API into the + * DSL, and provides all the functionality of the PAPI. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}, the + * processing progress can be observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3238,8 +3373,9 @@  KStream transform(final TransformerSupplier
-     * The second strategy is for the given {@link TransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link TransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyTransformerSupplier implements TransformerSupplier {
      *     // supply transformer
@@ -3263,12 +3399,11 @@  KStream transform(final TransformerSupplier
      * 

- * With either strategy, within the {@link Transformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link Transformer} must return a {@link KeyValue} type in {@link Transformer#transform(Object, Object) - * transform()}. - * The return value of {@link Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, + * With either strategy, within the {@link Transformer}, the state is obtained via the {@link + * ProcessorContext}. To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * punctuate()}, a schedule must be registered. The {@link Transformer} must return a {@link + * KeyValue} type in {@link Transformer#transform(Object, Object) transform()}. The return value + * of {@link Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, * in which case no record is emitted. *

{@code
      * class MyTransformer implements Transformer {
@@ -3292,65 +3427,72 @@  KStream transform(final TransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before {@code transform()}.
-     * 

- * Transforming records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@code KStream}. - * (cf. {@link #transformValues(ValueTransformerSupplier, String...) transformValues()} ) - *

- * Note that it is possible to emit multiple records for each input record by using - * {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} in - * {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be - * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should - * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * If in {@link Transformer#transform(Object, Object) Transformer#transform()} multiple records need to be emitted - * for each input record, it is recommended to use {@link #flatTransform(TransformerSupplier, String...) - * flatTransform()}. - * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating - * a single {@link Transformer} object and returning the same object reference in {@link TransformerSupplier#get()} would be - * a violation of the supplier pattern and leads to runtime exceptions. - * - * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly constructed - * {@link Transformer} + * Even if any upstream operation was key-changing, no auto-repartition is triggered. If + * repartitioning is required, a call to {@link #repartition()} should be performed before + * {@code transform()}. + *

+ * Transforming records might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #transformValues(ValueTransformerSupplier, String...) transformValues()} ) + *

+ * Note that it is possible to emit multiple records for each input record by using {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} in + * {@link Transformer#transform(Object, Object) Transformer#transform()} and {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. Be + * aware that a mismatch between the types of the emitted records and the type of the stream + * would only be detected at runtime. To ensure type-safety at compile-time, {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} + * should not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} + * and {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()}. If in {@link Transformer#transform(Object, Object) + * Transformer#transform()} multiple records need to be emitted for each input record, it is + * recommended to use {@link #flatTransform(TransformerSupplier, String...) flatTransform()}. + * The supplier should always generate a new instance each time {@link + * TransformerSupplier#get()} gets called. Creating a single {@link Transformer} object and + * returning the same object reference in {@link TransformerSupplier#get()} would be a violation + * of the supplier pattern and leads to runtime exceptions. + * + * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly + * constructed {@link Transformer} * @param named a {@link Named} config used to name the processor in the topology - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the key type of the new stream - * @param the value type of the new stream - * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) + * @param stateStoreNames the names of the state stores used by the processor; not required + * if the supplier implements {@link ConnectedStoreProvider#stores()} + * @param the key type of the new stream + * @param the value type of the new stream + * @return a {@code KStream} that contains more or less records with new key and value (possibly + * of different type) * @see #map(KeyValueMapper) * @see #flatTransform(TransformerSupplier, String...) * @see #transformValues(ValueTransformerSupplier, String...) * @see #transformValues(ValueTransformerWithKeySupplier, String...) * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ - KStream transform(final TransformerSupplier> transformerSupplier, - final Named named, - final String... stateStoreNames); + KStream transform( + final TransformerSupplier> transformerSupplier, + final Named named, + final String... stateStoreNames); /** - * Transform each record of the input stream into zero or more records in the output stream (both key and value type - * can be altered arbitrarily). - * A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and - * returns zero or more output records. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #flatMap(KeyValueMapper) flatMap()} - * but allows access to the {@code ProcessorContext} and record metadata. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()} - * the processing progress can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform each record of the input stream into zero or more records in the output stream + * (both key and value type can be altered arbitrarily). A {@link Transformer} (provided by the + * given {@link TransformerSupplier}) is applied to each input record and returns zero or more + * output records. Thus, an input record {@code } can be transformed into output records + * {@code , , ...}. Attaching a state store makes this a stateful + * record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()}). If you choose + * not to attach one, this operation is similar to the stateless {@link #flatMap(KeyValueMapper) + * flatMap()} but allows access to the {@code ProcessorContext} and record metadata. + * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()} the processing progress can be observed and additional periodic + * actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3366,8 +3508,9 @@  KStream transform(final TransformerSupplier
-     * The second strategy is for the given {@link TransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link TransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyTransformerSupplier implements TransformerSupplier {
      *     // supply transformer
@@ -3391,13 +3534,14 @@  KStream transform(final TransformerSupplier
      * 

- * With either strategy, within the {@link Transformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link Transformer} must return an {@link java.lang.Iterable} type (e.g., any {@link java.util.Collection} - * type) in {@link Transformer#transform(Object, Object) transform()}. - * The return value of {@link Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, - * which is equal to returning an empty {@link java.lang.Iterable Iterable}, i.e., no records are emitted. + * With either strategy, within the {@link Transformer}, the state is obtained via the {@link + * ProcessorContext}. To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * punctuate()}, a schedule must be registered. The {@link Transformer} must return an {@link + * java.lang.Iterable} type (e.g., any {@link java.util.Collection} type) in {@link + * Transformer#transform(Object, Object) transform()}. The return value of {@link + * Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, which is + * equal to returning an empty {@link java.lang.Iterable Iterable}, i.e., no records are + * emitted. *

{@code
      * class MyTransformer implements Transformer {
      *     private ProcessorContext context;
@@ -3424,59 +3568,66 @@  KStream transform(final TransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code flatTransform()}.
      * 

- * Transforming records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@code KStream}. - * (cf. {@link #transformValues(ValueTransformerSupplier, String...) transformValues()}) + * Transforming records might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #transformValues(ValueTransformerSupplier, String...) transformValues()}) *

* Note that it is possible to emit records by using {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) - * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be - * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should - * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating - * a single {@link Transformer} object and returning the same object reference in {@link TransformerSupplier#get()} would be - * a violation of the supplier pattern and leads to runtime exceptions. - * - * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly constructed {@link Transformer} - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the key type of the new stream - * @param the value type of the new stream - * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) + * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} + * and {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()}. Be aware that a mismatch between the types of the emitted records + * and the type of the stream would only be detected at runtime. To ensure type-safety at + * compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * context#forward()} should not be used in {@link Transformer#transform(Object, Object) + * Transformer#transform()} and {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()}. The supplier should always generate a new instance each time {@link + * TransformerSupplier#get()} gets called. Creating a single {@link Transformer} object and + * returning the same object reference in {@link TransformerSupplier#get()} would be a violation + * of the supplier pattern and leads to runtime exceptions. + * + * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly + * constructed {@link Transformer} + * @param stateStoreNames the names of the state stores used by the processor; not required + * if the supplier implements {@link ConnectedStoreProvider#stores()} + * @param the key type of the new stream + * @param the value type of the new stream + * @return a {@code KStream} that contains more or less records with new key and value (possibly + * of different type) * @see #flatMap(KeyValueMapper) * @see #transform(TransformerSupplier, String...) * @see #transformValues(ValueTransformerSupplier, String...) * @see #transformValues(ValueTransformerWithKeySupplier, String...) * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ - KStream flatTransform(final TransformerSupplier>> transformerSupplier, - final String... stateStoreNames); + KStream flatTransform( + final TransformerSupplier>> transformerSupplier, + final String... stateStoreNames); /** - * Transform each record of the input stream into zero or more records in the output stream (both key and value type - * can be altered arbitrarily). - * A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and - * returns zero or more output records. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #flatMap(KeyValueMapper) flatMap()} - * but allows access to the {@code ProcessorContext} and record metadata. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()} - * the processing progress can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform each record of the input stream into zero or more records in the output stream + * (both key and value type can be altered arbitrarily). A {@link Transformer} (provided by the + * given {@link TransformerSupplier}) is applied to each input record and returns zero or more + * output records. Thus, an input record {@code } can be transformed into output records + * {@code , , ...}. Attaching a state store makes this a stateful + * record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()}). If you choose + * not to attach one, this operation is similar to the stateless {@link #flatMap(KeyValueMapper) + * flatMap()} but allows access to the {@code ProcessorContext} and record metadata. + * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()} the processing progress can be observed and additional periodic + * actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3492,8 +3643,9 @@  KStream flatTransform(final TransformerSupplier
-     * The second strategy is for the given {@link TransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link TransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyTransformerSupplier implements TransformerSupplier {
      *     // supply transformer
@@ -3517,13 +3669,14 @@  KStream flatTransform(final TransformerSupplier
      * 

- * With either strategy, within the {@link Transformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link Transformer} must return an {@link java.lang.Iterable} type (e.g., any {@link java.util.Collection} - * type) in {@link Transformer#transform(Object, Object) transform()}. - * The return value of {@link Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, - * which is equal to returning an empty {@link java.lang.Iterable Iterable}, i.e., no records are emitted. + * With either strategy, within the {@link Transformer}, the state is obtained via the {@link + * ProcessorContext}. To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * punctuate()}, a schedule must be registered. The {@link Transformer} must return an {@link + * java.lang.Iterable} type (e.g., any {@link java.util.Collection} type) in {@link + * Transformer#transform(Object, Object) transform()}. The return value of {@link + * Transformer#transform(Object, Object) Transformer#transform()} may be {@code null}, which is + * equal to returning an empty {@link java.lang.Iterable Iterable}, i.e., no records are + * emitted. *

{@code
      * class MyTransformer implements Transformer {
      *     private ProcessorContext context;
@@ -3550,60 +3703,67 @@  KStream flatTransform(final TransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code flatTransform()}.
      * 

- * Transforming records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@code KStream}. - * (cf. {@link #transformValues(ValueTransformerSupplier, String...) transformValues()}) + * Transforming records might result in an internal data redistribution if a key based operator + * (like an aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #transformValues(ValueTransformerSupplier, String...) transformValues()}) *

* Note that it is possible to emit records by using {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) - * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be - * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) context#forward()} should - * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and - * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. - * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating - * a single {@link Transformer} object and returning the same object reference in {@link TransformerSupplier#get()} would be - * a violation of the supplier pattern and leads to runtime exceptions. - * - * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly constructed {@link Transformer} + * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} + * and {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()}. Be aware that a mismatch between the types of the emitted records + * and the type of the stream would only be detected at runtime. To ensure type-safety at + * compile-time, {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * context#forward()} should not be used in {@link Transformer#transform(Object, Object) + * Transformer#transform()} and {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * Punctuator#punctuate()}. The supplier should always generate a new instance each time {@link + * TransformerSupplier#get()} gets called. Creating a single {@link Transformer} object and + * returning the same object reference in {@link TransformerSupplier#get()} would be a violation + * of the supplier pattern and leads to runtime exceptions. + * + * @param transformerSupplier an instance of {@link TransformerSupplier} that generates a newly + * constructed {@link Transformer} * @param named a {@link Named} config used to name the processor in the topology - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the key type of the new stream - * @param the value type of the new stream - * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) + * @param stateStoreNames the names of the state stores used by the processor; not required + * if the supplier implements {@link ConnectedStoreProvider#stores()} + * @param the key type of the new stream + * @param the value type of the new stream + * @return a {@code KStream} that contains more or less records with new key and value (possibly + * of different type) * @see #flatMap(KeyValueMapper) * @see #transform(TransformerSupplier, String...) * @see #transformValues(ValueTransformerSupplier, String...) * @see #transformValues(ValueTransformerWithKeySupplier, String...) * @see #process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) */ - KStream flatTransform(final TransformerSupplier>> transformerSupplier, - final Named named, - final String... stateStoreNames); + KStream flatTransform( + final TransformerSupplier>> transformerSupplier, + final Named named, + final String... stateStoreNames); /** - * Transform the value of each input record into a new value (with possibly a new type) of the output record. - * A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input - * record value and computes a new value for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress - * can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform the value of each input record into a new value (with possibly a new type) of the + * output record. A {@link ValueTransformer} (provided by the given {@link + * ValueTransformerSupplier}) is applied to each input record value and computes a new value for + * it. Thus, an input record {@code } can be transformed into an output record {@code + * }. Attaching a state store makes this a stateful record-by-record operation (cf. {@link + * #mapValues(ValueMapper) mapValues()}). If you choose not to attach one, this operation is + * similar to the stateless {@link #mapValues(ValueMapper) mapValues()} but allows access to the + * {@code ProcessorContext} and record metadata. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be + * observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3619,8 +3779,9 @@  KStream flatTransform(final TransformerSupplier
-     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerSupplier implements ValueTransformerSupplier {
      *     // supply transformer
@@ -3644,14 +3805,15 @@  KStream flatTransform(final TransformerSupplier
      * 

- * With either strategy, within the {@link ValueTransformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformer} must return the new value in {@link ValueTransformer#transform(Object) transform()}. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()}, no additional {@link KeyValue} - * pairs can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformer} tries to - * emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformer}, the state is obtained via the + * {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformer} must return the new value in {@link + * ValueTransformer#transform(Object) transform()}. In contrast to {@link + * #transform(TransformerSupplier, String...) transform()}, no additional {@link KeyValue} pairs + * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformer} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformer implements ValueTransformer {
      *     private StateStore state;
@@ -3672,46 +3834,56 @@  KStream flatTransform(final TransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code transformValues()}.
      * 

- * Setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #transform(TransformerSupplier, String...)}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that generates a newly constructed {@link ValueTransformer} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformer} object - * and returning the same object reference in {@link ValueTransformer} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type) + * Setting a new value preserves data co-location with respect to the key. Thus, no + * internal data redistribution is required if a key based operator (like an aggregation or + * join) is applied to the result {@code KStream}. (cf. {@link #transform(TransformerSupplier, + * String...)}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that + * generates a newly constructed {@link ValueTransformer} The + * supplier should always generate a new instance. Creating a + * single {@link ValueTransformer} object and returning the same + * object reference in {@link ValueTransformer} is a violation + * of the supplier pattern and leads to runtime exceptions. + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains records with unmodified key and new values (possibly + * of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) */ - KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, - final String... stateStoreNames); + KStream transformValues( + final ValueTransformerSupplier valueTransformerSupplier, + final String... stateStoreNames); + /** - * Transform the value of each input record into a new value (with possibly a new type) of the output record. - * A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input - * record value and computes a new value for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress - * can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform the value of each input record into a new value (with possibly a new type) of the + * output record. A {@link ValueTransformer} (provided by the given {@link + * ValueTransformerSupplier}) is applied to each input record value and computes a new value for + * it. Thus, an input record {@code } can be transformed into an output record {@code + * }. Attaching a state store makes this a stateful record-by-record operation (cf. {@link + * #mapValues(ValueMapper) mapValues()}). If you choose not to attach one, this operation is + * similar to the stateless {@link #mapValues(ValueMapper) mapValues()} but allows access to the + * {@code ProcessorContext} and record metadata. This is essentially mixing the Processor API + * into the DSL, and provides all the functionality of the PAPI. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be + * observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3727,8 +3899,9 @@  KStream transformValues(final ValueTransformerSupplier
-     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerSupplier implements ValueTransformerSupplier {
      *     // supply transformer
@@ -3752,14 +3925,15 @@  KStream transformValues(final ValueTransformerSupplier
      * 

- * With either strategy, within the {@link ValueTransformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformer} must return the new value in {@link ValueTransformer#transform(Object) transform()}. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()}, no additional {@link KeyValue} - * pairs can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformer} tries to - * emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformer}, the state is obtained via the + * {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformer} must return the new value in {@link + * ValueTransformer#transform(Object) transform()}. In contrast to {@link + * #transform(TransformerSupplier, String...) transform()}, no additional {@link KeyValue} pairs + * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformer} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformer implements ValueTransformer {
      *     private StateStore state;
@@ -3780,49 +3954,59 @@  KStream transformValues(final ValueTransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code transformValues()}.
      * 

- * Setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #transform(TransformerSupplier, String...)}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that generates a newly constructed {@link ValueTransformer} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformer} object - * and returning the same object reference in {@link ValueTransformer} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param named a {@link Named} config used to name the processor in the topology - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type) + * Setting a new value preserves data co-location with respect to the key. Thus, no + * internal data redistribution is required if a key based operator (like an aggregation or + * join) is applied to the result {@code KStream}. (cf. {@link #transform(TransformerSupplier, + * String...)}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that + * generates a newly constructed {@link ValueTransformer} The + * supplier should always generate a new instance. Creating a + * single {@link ValueTransformer} object and returning the same + * object reference in {@link ValueTransformer} is a violation + * of the supplier pattern and leads to runtime exceptions. + * @param named a {@link Named} config used to name the processor in the + * topology + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains records with unmodified key and new values (possibly + * of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) */ - KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, - final Named named, - final String... stateStoreNames); + KStream transformValues( + final ValueTransformerSupplier valueTransformerSupplier, + final Named named, + final String... stateStoreNames); /** - * Transform the value of each input record into a new value (with possibly a new type) of the output record. - * A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to - * each input record value and computes a new value for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapperWithKey) mapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapperWithKey) mapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress - * can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform the value of each input record into a new value (with possibly a new type) of the + * output record. A {@link ValueTransformerWithKey} (provided by the given {@link + * ValueTransformerWithKeySupplier}) is applied to each input record value and computes a new + * value for it. Thus, an input record {@code } can be transformed into an output record + * {@code }. Attaching a state store makes this a stateful record-by-record operation (cf. + * {@link #mapValues(ValueMapperWithKey) mapValues()}). If you choose not to attach one, this + * operation is similar to the stateless {@link #mapValues(ValueMapperWithKey) mapValues()} but + * allows access to the {@code ProcessorContext} and record metadata. This is essentially mixing + * the Processor API into the DSL, and provides all the functionality of the PAPI. Furthermore, + * via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing + * progress can be observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3838,8 +4022,9 @@  KStream transformValues(final ValueTransformerSupplier
-     * The second strategy is for the given {@link ValueTransformerWithKeySupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerWithKeySupplier} to implement
+     * {@link ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerWithKeySupplier implements ValueTransformerWithKeySupplier {
      *     // supply transformer
@@ -3863,16 +4048,16 @@  KStream transformValues(final ValueTransformerSupplier
      * 

- * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformerWithKey} must return the new value in - * {@link ValueTransformerWithKey#transform(Object, Object) transform()}. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()} and - * {@link #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link KeyValue} pairs - * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformerWithKey} tries - * to emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via + * the {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformerWithKey} must return the new value in {@link + * ValueTransformerWithKey#transform(Object, Object) transform()}. In contrast to {@link + * #transform(TransformerSupplier, String...) transform()} and {@link + * #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link + * KeyValue} pairs can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformerWithKey} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformerWithKey implements ValueTransformerWithKey {
      *     private StateStore state;
@@ -3893,48 +4078,58 @@  KStream transformValues(final ValueTransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code transformValues()}.
      * 

- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #transform(TransformerSupplier, String...)}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that generates a newly constructed {@link ValueTransformerWithKey} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformerWithKey} object - * and returning the same object reference in {@link ValueTransformerWithKey} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, setting a new value preserves data co-location with respect to the key. + * Thus, no internal data redistribution is required if a key based operator (like an + * aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #transform(TransformerSupplier, String...)}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that + * generates a newly constructed {@link ValueTransformerWithKey} + * The supplier should always generate a new instance. Creating + * a single {@link ValueTransformerWithKey} object and returning + * the same object reference in {@link ValueTransformerWithKey} + * is a violation of the supplier pattern and leads to runtime + * exceptions. + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains records with unmodified key and new values (possibly + * of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) */ - KStream transformValues(final ValueTransformerWithKeySupplier valueTransformerSupplier, - final String... stateStoreNames); + KStream transformValues( + final ValueTransformerWithKeySupplier valueTransformerSupplier, + final String... stateStoreNames); /** - * Transform the value of each input record into a new value (with possibly a new type) of the output record. - * A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to - * each input record value and computes a new value for it. - * Thus, an input record {@code } can be transformed into an output record {@code }. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapperWithKey) mapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapperWithKey) mapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress - * can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * Transform the value of each input record into a new value (with possibly a new type) of the + * output record. A {@link ValueTransformerWithKey} (provided by the given {@link + * ValueTransformerWithKeySupplier}) is applied to each input record value and computes a new + * value for it. Thus, an input record {@code } can be transformed into an output record + * {@code }. Attaching a state store makes this a stateful record-by-record operation (cf. + * {@link #mapValues(ValueMapperWithKey) mapValues()}). If you choose not to attach one, this + * operation is similar to the stateless {@link #mapValues(ValueMapperWithKey) mapValues()} but + * allows access to the {@code ProcessorContext} and record metadata. This is essentially mixing + * the Processor API into the DSL, and provides all the functionality of the PAPI. Furthermore, + * via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing + * progress can be observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -3950,8 +4145,9 @@  KStream transformValues(final ValueTransformerWithKeySupplier
-     * The second strategy is for the given {@link ValueTransformerWithKeySupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerWithKeySupplier} to implement
+     * {@link ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerWithKeySupplier implements ValueTransformerWithKeySupplier {
      *     // supply transformer
@@ -3975,16 +4171,16 @@  KStream transformValues(final ValueTransformerWithKeySupplier
      * 

- * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformerWithKey} must return the new value in - * {@link ValueTransformerWithKey#transform(Object, Object) transform()}. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()} and - * {@link #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link KeyValue} pairs - * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformerWithKey} tries - * to emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via + * the {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformerWithKey} must return the new value in {@link + * ValueTransformerWithKey#transform(Object, Object) transform()}. In contrast to {@link + * #transform(TransformerSupplier, String...) transform()} and {@link + * #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link + * KeyValue} pairs can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformerWithKey} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformerWithKey implements ValueTransformerWithKey {
      *     private StateStore state;
@@ -4005,50 +4201,62 @@  KStream transformValues(final ValueTransformerWithKeySupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code transformValues()}.
      * 

- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #transform(TransformerSupplier, String...)}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that generates a newly constructed {@link ValueTransformerWithKey} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformerWithKey} object - * and returning the same object reference in {@link ValueTransformerWithKey} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param named a {@link Named} config used to name the processor in the topology - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, setting a new value preserves data co-location with respect to the key. + * Thus, no internal data redistribution is required if a key based operator (like an + * aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #transform(TransformerSupplier, String...)}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that + * generates a newly constructed {@link ValueTransformerWithKey} + * The supplier should always generate a new instance. Creating + * a single {@link ValueTransformerWithKey} object and returning + * the same object reference in {@link ValueTransformerWithKey} + * is a violation of the supplier pattern and leads to runtime + * exceptions. + * @param named a {@link Named} config used to name the processor in the + * topology + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains records with unmodified key and new values (possibly + * of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) */ - KStream transformValues(final ValueTransformerWithKeySupplier valueTransformerSupplier, - final Named named, - final String... stateStoreNames); + KStream transformValues( + final ValueTransformerWithKeySupplier valueTransformerSupplier, + final Named named, + final String... stateStoreNames); + /** * Transform the value of each input record into zero or more new values (with possibly a new - * type) and emit for each new value a record with the same key of the input record and the value. - * A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input - * record value and computes zero or more new values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()} - * the processing progress can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * type) and emit for each new value a record with the same key of the input record and the + * value. A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is + * applied to each input record value and computes zero or more new values. Thus, an input + * record {@code } can be transformed into output records {@code , , ...}. + * Attaching a state store makes this a stateful record-by-record operation (cf. {@link + * #mapValues(ValueMapper) mapValues()}). If you choose not to attach one, this operation is + * similar to the stateless {@link #mapValues(ValueMapper) mapValues()} but allows access to the + * {@code ProcessorContext} and record metadata. This is essentially mixing the Processor API + * into the DSL, and provides all the functionality of the PAPI. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()} the + * processing progress can be observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -4064,8 +4272,9 @@  KStream transformValues(final ValueTransformerWithKeySupplier
-     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerSupplier implements ValueTransformerSupplier {
      *     // supply transformer
@@ -4089,19 +4298,19 @@  KStream transformValues(final ValueTransformerWithKeySupplier
      * 

- * With either strategy, within the {@link ValueTransformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformer} must return an {@link java.lang.Iterable} type (e.g., any - * {@link java.util.Collection} type) in {@link ValueTransformer#transform(Object) - * transform()}. - * If the return value of {@link ValueTransformer#transform(Object) ValueTransformer#transform()} is an empty - * {@link java.lang.Iterable Iterable} or {@code null}, no records are emitted. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()} and - * {@link #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link KeyValue} pairs - * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformer} tries to - * emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformer}, the state is obtained via the + * {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformer} must return an {@link java.lang.Iterable} type + * (e.g., any {@link java.util.Collection} type) in {@link ValueTransformer#transform(Object) + * transform()}. If the return value of {@link ValueTransformer#transform(Object) + * ValueTransformer#transform()} is an empty {@link java.lang.Iterable Iterable} or {@code + * null}, no records are emitted. In contrast to {@link #transform(TransformerSupplier, + * String...) transform()} and {@link #flatTransform(TransformerSupplier, String...) + * flatTransform()}, no additional {@link KeyValue} pairs can be emitted via {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformer} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformer implements ValueTransformer {
      *     private StateStore state;
@@ -4126,51 +4335,58 @@  KStream transformValues(final ValueTransformerWithKeySupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code flatTransformValues()}.
      * 

- * Setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatTransform(TransformerSupplier, String...) - * flatTransform()}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that generates a newly constructed {@link ValueTransformer} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformer} object - * and returning the same object reference in {@link ValueTransformer} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified key and new values (possibly of - * different type) + * Setting a new value preserves data co-location with respect to the key. Thus, no + * internal data redistribution is required if a key based operator (like an aggregation or + * join) is applied to the result {@code KStream}. (cf. {@link #flatTransform(TransformerSupplier, + * String...) flatTransform()}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that + * generates a newly constructed {@link ValueTransformer} The + * supplier should always generate a new instance. Creating a + * single {@link ValueTransformer} object and returning the same + * object reference in {@link ValueTransformer} is a violation + * of the supplier pattern and leads to runtime exceptions. + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified key and new + * values (possibly of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) * @see #flatTransform(TransformerSupplier, String...) */ - KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, - final String... stateStoreNames); + KStream flatTransformValues( + final ValueTransformerSupplier> valueTransformerSupplier, + final String... stateStoreNames); /** * Transform the value of each input record into zero or more new values (with possibly a new - * type) and emit for each new value a record with the same key of the input record and the value. - * A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input - * record value and computes zero or more new values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()} - * the processing progress can be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * type) and emit for each new value a record with the same key of the input record and the + * value. A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is + * applied to each input record value and computes zero or more new values. Thus, an input + * record {@code } can be transformed into output records {@code , , ...}. + * Attaching a state store makes this a stateful record-by-record operation (cf. {@link + * #mapValues(ValueMapper) mapValues()}). If you choose not to attach one, this operation is + * similar to the stateless {@link #mapValues(ValueMapper) mapValues()} but allows access to the + * {@code ProcessorContext} and record metadata. This is essentially mixing the Processor API + * into the DSL, and provides all the functionality of the PAPI. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()} the + * processing progress can be observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -4186,8 +4402,9 @@  KStream flatTransformValues(final ValueTransformerSupplier
-     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerSupplier implements ValueTransformerSupplier {
      *     // supply transformer
@@ -4211,19 +4428,19 @@  KStream flatTransformValues(final ValueTransformerSupplier
      * 

- * With either strategy, within the {@link ValueTransformer}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformer} must return an {@link java.lang.Iterable} type (e.g., any - * {@link java.util.Collection} type) in {@link ValueTransformer#transform(Object) - * transform()}. - * If the return value of {@link ValueTransformer#transform(Object) ValueTransformer#transform()} is an empty - * {@link java.lang.Iterable Iterable} or {@code null}, no records are emitted. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()} and - * {@link #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link KeyValue} pairs - * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformer} tries to - * emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformer}, the state is obtained via the + * {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformer} must return an {@link java.lang.Iterable} type + * (e.g., any {@link java.util.Collection} type) in {@link ValueTransformer#transform(Object) + * transform()}. If the return value of {@link ValueTransformer#transform(Object) + * ValueTransformer#transform()} is an empty {@link java.lang.Iterable Iterable} or {@code + * null}, no records are emitted. In contrast to {@link #transform(TransformerSupplier, + * String...) transform()} and {@link #flatTransform(TransformerSupplier, String...) + * flatTransform()}, no additional {@link KeyValue} pairs can be emitted via {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformer} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformer implements ValueTransformer {
      *     private StateStore state;
@@ -4248,53 +4465,62 @@  KStream flatTransformValues(final ValueTransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code flatTransformValues()}.
      * 

- * Setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatTransform(TransformerSupplier, String...) - * flatTransform()}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that generates a newly constructed {@link ValueTransformer} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformer} object - * and returning the same object reference in {@link ValueTransformer} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param named a {@link Named} config used to name the processor in the topology - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified key and new values (possibly of - * different type) + * Setting a new value preserves data co-location with respect to the key. Thus, no + * internal data redistribution is required if a key based operator (like an aggregation or + * join) is applied to the result {@code KStream}. (cf. {@link #flatTransform(TransformerSupplier, + * String...) flatTransform()}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerSupplier} that + * generates a newly constructed {@link ValueTransformer} The + * supplier should always generate a new instance. Creating a + * single {@link ValueTransformer} object and returning the same + * object reference in {@link ValueTransformer} is a violation + * of the supplier pattern and leads to runtime exceptions. + * @param named a {@link Named} config used to name the processor in the + * topology + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified key and new + * values (possibly of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) * @see #flatTransform(TransformerSupplier, String...) */ - KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, - final Named named, - final String... stateStoreNames); + KStream flatTransformValues( + final ValueTransformerSupplier> valueTransformerSupplier, + final Named named, + final String... stateStoreNames); /** * Transform the value of each input record into zero or more new values (with possibly a new - * type) and emit for each new value a record with the same key of the input record and the value. - * A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to - * each input record value and computes zero or more new values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #flatMapValues(ValueMapperWithKey) flatMapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can - * be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * type) and emit for each new value a record with the same key of the input record and the + * value. A {@link ValueTransformerWithKey} (provided by the given {@link + * ValueTransformerWithKeySupplier}) is applied to each input record value and computes zero or + * more new values. Thus, an input record {@code } can be transformed into output records + * {@code , , ...}. Attaching a state store makes this a stateful record-by-record + * operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}). If you choose not + * to attach one, this operation is similar to the stateless {@link + * #flatMapValues(ValueMapperWithKey) flatMapValues()} but allows access to the {@code + * ProcessorContext} and record metadata. This is essentially mixing the Processor API into the + * DSL, and provides all the functionality of the PAPI. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be + * observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -4310,8 +4536,9 @@  KStream flatTransformValues(final ValueTransformerSupplier
-     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerWithKeySupplier implements ValueTransformerWithKeySupplier {
      *     // supply transformer
@@ -4335,19 +4562,19 @@  KStream flatTransformValues(final ValueTransformerSupplier
      * 

- * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformerWithKey} must return an {@link java.lang.Iterable} type (e.g., any - * {@link java.util.Collection} type) in {@link ValueTransformerWithKey#transform(Object, Object) - * transform()}. - * If the return value of {@link ValueTransformerWithKey#transform(Object, Object) ValueTransformerWithKey#transform()} - * is an empty {@link java.lang.Iterable Iterable} or {@code null}, no records are emitted. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()} and - * {@link #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link KeyValue} pairs - * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformerWithKey} tries - * to emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via + * the {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformerWithKey} must return an {@link java.lang.Iterable} + * type (e.g., any {@link java.util.Collection} type) in {@link ValueTransformerWithKey#transform(Object, + * Object) transform()}. If the return value of {@link ValueTransformerWithKey#transform(Object, + * Object) ValueTransformerWithKey#transform()} is an empty {@link java.lang.Iterable Iterable} + * or {@code null}, no records are emitted. In contrast to {@link #transform(TransformerSupplier, + * String...) transform()} and {@link #flatTransform(TransformerSupplier, String...) + * flatTransform()}, no additional {@link KeyValue} pairs can be emitted via {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformerWithKey} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformerWithKey implements ValueTransformerWithKey {
      *     private StateStore state;
@@ -4372,52 +4599,61 @@  KStream flatTransformValues(final ValueTransformerSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code flatTransformValues()}.
      * 

- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatTransform(TransformerSupplier, String...) - * flatTransform()}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that generates a newly constructed {@link ValueTransformerWithKey} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformerWithKey} object - * and returning the same object reference in {@link ValueTransformerWithKey} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified key and new values (possibly of - * different type) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, setting a new value preserves data co-location with respect to the key. + * Thus, no internal data redistribution is required if a key based operator (like an + * aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #flatTransform(TransformerSupplier, String...) flatTransform()}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that + * generates a newly constructed {@link ValueTransformerWithKey} + * The supplier should always generate a new instance. Creating + * a single {@link ValueTransformerWithKey} object and returning + * the same object reference in {@link ValueTransformerWithKey} + * is a violation of the supplier pattern and leads to runtime + * exceptions. + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified key and new + * values (possibly of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) * @see #flatTransform(TransformerSupplier, String...) */ - KStream flatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerSupplier, - final String... stateStoreNames); + KStream flatTransformValues( + final ValueTransformerWithKeySupplier> valueTransformerSupplier, + final String... stateStoreNames); /** * Transform the value of each input record into zero or more new values (with possibly a new - * type) and emit for each new value a record with the same key of the input record and the value. - * A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to - * each input record value and computes zero or more new values. - * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}). - * If you choose not to attach one, this operation is similar to the stateless {@link #flatMapValues(ValueMapperWithKey) flatMapValues()} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can - * be observed and additional periodic actions can be performed. - *

- * In order for the transformer to use state stores, the stores must be added to the topology and connected to the - * transformer using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the transformer. + * type) and emit for each new value a record with the same key of the input record and the + * value. A {@link ValueTransformerWithKey} (provided by the given {@link + * ValueTransformerWithKeySupplier}) is applied to each input record value and computes zero or + * more new values. Thus, an input record {@code } can be transformed into output records + * {@code , , ...}. Attaching a state store makes this a stateful record-by-record + * operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}). If you choose not + * to attach one, this operation is similar to the stateless {@link + * #flatMapValues(ValueMapperWithKey) flatMapValues()} but allows access to the {@code + * ProcessorContext} and record metadata. This is essentially mixing the Processor API into the + * DSL, and provides all the functionality of the PAPI. Furthermore, via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be + * observed and additional periodic actions can be performed. + *

+ * In order for the transformer to use state stores, the stores must be added to the topology + * and connected to the transformer using at least one of two strategies (though it's not + * required to connect global state stores; read-only access to global state stores is available + * by default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the transformer. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -4433,8 +4669,9 @@  KStream flatTransformValues(final ValueTransformerWithKeySupplier
-     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the transformer.
+     * The second strategy is for the given {@link ValueTransformerSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the transformer.
      * 
{@code
      * class MyValueTransformerWithKeySupplier implements ValueTransformerWithKeySupplier {
      *     // supply transformer
@@ -4458,19 +4695,19 @@  KStream flatTransformValues(final ValueTransformerWithKeySupplier
      * 

- * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. - * The {@link ValueTransformerWithKey} must return an {@link java.lang.Iterable} type (e.g., any - * {@link java.util.Collection} type) in {@link ValueTransformerWithKey#transform(Object, Object) - * transform()}. - * If the return value of {@link ValueTransformerWithKey#transform(Object, Object) ValueTransformerWithKey#transform()} - * is an empty {@link java.lang.Iterable Iterable} or {@code null}, no records are emitted. - * In contrast to {@link #transform(TransformerSupplier, String...) transform()} and - * {@link #flatTransform(TransformerSupplier, String...) flatTransform()}, no additional {@link KeyValue} pairs - * can be emitted via {@link ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) ProcessorContext.forward()}. - * A {@link org.apache.kafka.streams.errors.StreamsException} is thrown if the {@link ValueTransformerWithKey} tries - * to emit a {@link KeyValue} pair. + * With either strategy, within the {@link ValueTransformerWithKey}, the state is obtained via + * the {@link ProcessorContext}. To trigger periodic actions via {@link + * org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, a schedule must + * be registered. The {@link ValueTransformerWithKey} must return an {@link java.lang.Iterable} + * type (e.g., any {@link java.util.Collection} type) in {@link ValueTransformerWithKey#transform(Object, + * Object) transform()}. If the return value of {@link ValueTransformerWithKey#transform(Object, + * Object) ValueTransformerWithKey#transform()} is an empty {@link java.lang.Iterable Iterable} + * or {@code null}, no records are emitted. In contrast to {@link #transform(TransformerSupplier, + * String...) transform()} and {@link #flatTransform(TransformerSupplier, String...) + * flatTransform()}, no additional {@link KeyValue} pairs can be emitted via {@link + * ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * ProcessorContext.forward()}. A {@link org.apache.kafka.streams.errors.StreamsException} is + * thrown if the {@link ValueTransformerWithKey} tries to emit a {@link KeyValue} pair. *

{@code
      * class MyValueTransformerWithKey implements ValueTransformerWithKey {
      *     private StateStore state;
@@ -4495,52 +4732,60 @@  KStream flatTransformValues(final ValueTransformerWithKeySupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
      * {@code flatTransformValues()}.
      * 

- * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. - * So, setting a new value preserves data co-location with respect to the key. - * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@code KStream}. (cf. {@link #flatTransform(TransformerSupplier, String...) - * flatTransform()}) - * - * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that generates a newly constructed {@link ValueTransformerWithKey} - * The supplier should always generate a new instance. Creating a single {@link ValueTransformerWithKey} object - * and returning the same object reference in {@link ValueTransformerWithKey} is a - * violation of the supplier pattern and leads to runtime exceptions. - * @param named a {@link Named} config used to name the processor in the topology - * @param stateStoreNames the names of the state stores used by the processor; not required if the supplier - * implements {@link ConnectedStoreProvider#stores()} - * @param the value type of the result stream - * @return a {@code KStream} that contains more or less records with unmodified key and new values (possibly of - * different type) + * Note that the key is read-only and should not be modified, as this can lead to corrupt + * partitioning. So, setting a new value preserves data co-location with respect to the key. + * Thus, no internal data redistribution is required if a key based operator (like an + * aggregation or join) is applied to the result {@code KStream}. (cf. {@link + * #flatTransform(TransformerSupplier, String...) flatTransform()}) + * + * @param valueTransformerSupplier an instance of {@link ValueTransformerWithKeySupplier} that + * generates a newly constructed {@link ValueTransformerWithKey} + * The supplier should always generate a new instance. Creating + * a single {@link ValueTransformerWithKey} object and returning + * the same object reference in {@link ValueTransformerWithKey} + * is a violation of the supplier pattern and leads to runtime + * exceptions. + * @param named a {@link Named} config used to name the processor in the + * topology + * @param stateStoreNames the names of the state stores used by the processor; not + * required if the supplier implements {@link + * ConnectedStoreProvider#stores()} + * @param the value type of the result stream + * @return a {@code KStream} that contains more or less records with unmodified key and new + * values (possibly of different type) * @see #mapValues(ValueMapper) * @see #mapValues(ValueMapperWithKey) * @see #transform(TransformerSupplier, String...) * @see #flatTransform(TransformerSupplier, String...) */ - KStream flatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerSupplier, - final Named named, - final String... stateStoreNames); + KStream flatTransformValues( + final ValueTransformerWithKeySupplier> valueTransformerSupplier, + final Named named, + final String... stateStoreNames); /** - * Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given - * {@link ProcessorSupplier}). - * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}). - * If you choose not to attach one, this operation is similar to the stateless {@link #foreach(ForeachAction)} - * but allows access to the {@code ProcessorContext} and record metadata. - * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI. - * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress - * can be observed and additional periodic actions can be performed. - * Note that this is a terminal operation that returns void. - *

- * In order for the processor to use state stores, the stores must be added to the topology and connected to the - * processor using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the processor. + * Process all records in this stream, one record at a time, by applying a {@link Processor} + * (provided by the given {@link ProcessorSupplier}). Attaching a state store makes this a + * stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}). If you choose not + * to attach one, this operation is similar to the stateless {@link #foreach(ForeachAction)} but + * allows access to the {@code ProcessorContext} and record metadata. This is essentially mixing + * the Processor API into the DSL, and provides all the functionality of the PAPI. Furthermore, + * via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing + * progress can be observed and additional periodic actions can be performed. Note that this is + * a terminal operation that returns void. + *

+ * In order for the processor to use state stores, the stores must be added to the topology and + * connected to the processor using at least one of two strategies (though it's not required to + * connect global state stores; read-only access to global state stores is available by + * default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the processor. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -4556,8 +4801,9 @@  KStream flatTransformValues(final ValueTransformerWithKeySupplier
-     * The second strategy is for the given {@link ProcessorSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor.
+     * The second strategy is for the given {@link ProcessorSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the processor.
      * 
{@code
      * class MyProcessorSupplier implements ProcessorSupplier {
      *     // supply processor
@@ -4581,9 +4827,9 @@  KStream flatTransformValues(final ValueTransformerWithKeySupplier
      * 

- * With either strategy, within the {@link Processor}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. + * With either strategy, within the {@link Processor}, the state is obtained via the {@link + * ProcessorContext}. To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * punctuate()}, a schedule must be registered. *

{@code
      * class MyProcessor implements Processor {
      *     private StateStore state;
@@ -4603,38 +4849,44 @@  KStream flatTransformValues(final ValueTransformerWithKeySupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before {@code process()}.
-     *
-     * @param processorSupplier an instance of {@link ProcessorSupplier} that generates a newly constructed {@link Processor}
-     *                          The supplier should always generate a new instance. Creating a single {@link Processor} object
-     *                          and returning the same object reference in {@link ProcessorSupplier#get()} is a
-     *                          violation of the supplier pattern and leads to runtime exceptions.
-     * @param stateStoreNames     the names of the state stores used by the processor; not required if the supplier
-     *                            implements {@link ConnectedStoreProvider#stores()}
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
+     * {@code process()}.
+     *
+     * @param processorSupplier an instance of {@link ProcessorSupplier} that generates a newly
+     *                          constructed {@link Processor} The supplier should always generate a
+     *                          new instance. Creating a single {@link Processor} object and
+     *                          returning the same object reference in {@link ProcessorSupplier#get()}
+     *                          is a violation of the supplier pattern and leads to runtime
+     *                          exceptions.
+     * @param stateStoreNames   the names of the state stores used by the processor; not required if
+     *                          the supplier implements {@link ConnectedStoreProvider#stores()}
      * @see #foreach(ForeachAction)
      * @see #transform(TransformerSupplier, String...)
      */
-    void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier,
-                 final String... stateStoreNames);
+    void process(
+        final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier,
+        final String... stateStoreNames);
 
     /**
-     * Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given
-     * {@link ProcessorSupplier}).
-     * Attaching a state store makes this a stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}).
-     * If you choose not to attach one, this operation is similar to the stateless {@link #foreach(ForeachAction)}
-     * but allows access to the {@code ProcessorContext} and record metadata.
-     * This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
-     * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress
-     * can be observed and additional periodic actions can be performed.
-     * Note that this is a terminal operation that returns void.
-     * 

- * In order for the processor to use state stores, the stores must be added to the topology and connected to the - * processor using at least one of two strategies (though it's not required to connect global state stores; read-only - * access to global state stores is available by default). - *

- * The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)}, - * and specify the store names via {@code stateStoreNames} so they will be connected to the processor. + * Process all records in this stream, one record at a time, by applying a {@link Processor} + * (provided by the given {@link ProcessorSupplier}). Attaching a state store makes this a + * stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}). If you choose not + * to attach one, this operation is similar to the stateless {@link #foreach(ForeachAction)} but + * allows access to the {@code ProcessorContext} and record metadata. This is essentially mixing + * the Processor API into the DSL, and provides all the functionality of the PAPI. Furthermore, + * via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing + * progress can be observed and additional periodic actions can be performed. Note that this is + * a terminal operation that returns void. + *

+ * In order for the processor to use state stores, the stores must be added to the topology and + * connected to the processor using at least one of two strategies (though it's not required to + * connect global state stores; read-only access to global state stores is available by + * default). + *

+ * The first strategy is to manually add the {@link StoreBuilder}s via {@link + * Topology#addStateStore(StoreBuilder, String...)}, and specify the store names via {@code + * stateStoreNames} so they will be connected to the processor. *

{@code
      * // create store
      * StoreBuilder> keyValueStoreBuilder =
@@ -4650,8 +4902,9 @@ void process(final org.apache.kafka.streams.processor.ProcessorSupplier
-     * The second strategy is for the given {@link ProcessorSupplier} to implement {@link ConnectedStoreProvider#stores()},
-     * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor.
+     * The second strategy is for the given {@link ProcessorSupplier} to implement {@link
+     * ConnectedStoreProvider#stores()}, which provides the {@link StoreBuilder}s to be
+     * automatically added to the topology and connected to the processor.
      * 
{@code
      * class MyProcessorSupplier implements ProcessorSupplier {
      *     // supply processor
@@ -4675,9 +4928,9 @@ void process(final org.apache.kafka.streams.processor.ProcessorSupplier
      * 

- * With either strategy, within the {@link Processor}, the state is obtained via the {@link ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, - * a schedule must be registered. + * With either strategy, within the {@link Processor}, the state is obtained via the {@link + * ProcessorContext}. To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) + * punctuate()}, a schedule must be registered. *

{@code
      * class MyProcessor implements Processor {
      *     private StateStore state;
@@ -4697,20 +4950,24 @@ void process(final org.apache.kafka.streams.processor.ProcessorSupplier
-     * Even if any upstream operation was key-changing, no auto-repartition is triggered.
-     * If repartitioning is required, a call to {@link #repartition()} should be performed before {@code process()}.
-     *
-     * @param processorSupplier an instance of {@link ProcessorSupplier} that generates a newly constructed {@link Processor}
-     *                          The supplier should always generate a new instance. Creating a single {@link Processor} object
-     *                          and returning the same object reference in {@link ProcessorSupplier#get()} is a
-     *                          violation of the supplier pattern and leads to runtime exceptions.
+     * Even if any upstream operation was key-changing, no auto-repartition is triggered. If
+     * repartitioning is required, a call to {@link #repartition()} should be performed before
+     * {@code process()}.
+     *
+     * @param processorSupplier an instance of {@link ProcessorSupplier} that generates a newly
+     *                          constructed {@link Processor} The supplier should always generate a
+     *                          new instance. Creating a single {@link Processor} object and
+     *                          returning the same object reference in {@link ProcessorSupplier#get()}
+     *                          is a violation of the supplier pattern and leads to runtime
+     *                          exceptions.
      * @param named             a {@link Named} config used to name the processor in the topology
      * @param stateStoreNames   the names of the state store used by the processor
      * @see #foreach(ForeachAction)
      * @see #transform(TransformerSupplier, String...)
      */
-    void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier,
-                 final Named named,
-                 final String... stateStoreNames);
+    void process(
+        final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier,
+        final Named named,
+        final String... stateStoreNames);
 
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
index 04e55858bbc34..4d9fbf762f44b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import java.util.function.Function;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.streams.KafkaStreams;
@@ -30,8 +31,6 @@
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
 
-import java.util.function.Function;
-
 /**
  * {@code KTable} is an abstraction of a changelog stream from a primary-keyed table.
  * Each record in this changelog stream is an update on the primary-keyed table with the record key as the primary key.
@@ -341,10 +340,10 @@ KTable filterNot(final Predicate predicate,
      * delete the corresponding record in the result {@code KTable}.
      *
      * @param mapper a {@link ValueMapper} that computes a new output value
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapper mapper);
+     KTable mapValues(final ValueMapper mapper);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -372,11 +371,11 @@ KTable filterNot(final Predicate predicate,
      *
      * @param mapper a {@link ValueMapper} that computes a new output value
      * @param named  a {@link Named} config used to name the processor in the topology
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapper mapper,
-                                 final Named named);
+     KTable mapValues(final ValueMapper mapper,
+                                     final Named named);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -405,10 +404,11 @@  KTable mapValues(final ValueMapper mapper,
      * delete the corresponding record in the result {@code KTable}.
      *
      * @param mapper a {@link ValueMapperWithKey} that computes a new output value
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapperWithKey mapper);
+     KTable mapValues(
+        final ValueMapperWithKey mapper);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -438,11 +438,12 @@  KTable mapValues(final ValueMapper mapper,
      *
      * @param mapper a {@link ValueMapperWithKey} that computes a new output value
      * @param named  a {@link Named} config used to name the processor in the topology
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapperWithKey mapper,
-                                 final Named named);
+     KTable mapValues(
+        final ValueMapperWithKey mapper,
+        final Named named);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -483,12 +484,12 @@  KTable mapValues(final ValueMapperWithKey   the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      *
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapper mapper,
-                                 final Materialized> materialized);
+     KTable mapValues(final ValueMapper mapper,
+                                     final Materialized> materialized);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -530,13 +531,13 @@  KTable mapValues(final ValueMapper mapper,
      * @param named  a {@link Named} config used to name the processor in the topology
      * @param materialized  a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
      *                      should be materialized. Cannot be {@code null}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      *
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapper mapper,
-                                 final Named named,
-                                 final Materialized> materialized);
+     KTable mapValues(final ValueMapper mapper,
+                                     final Named named,
+                                     final Materialized> materialized);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -578,12 +579,13 @@  KTable mapValues(final ValueMapper mapper,
      * @param mapper a {@link ValueMapperWithKey} that computes a new output value
      * @param materialized  a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
      *                      should be materialized. Cannot be {@code null}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      *
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapperWithKey mapper,
-                                 final Materialized> materialized);
+     KTable mapValues(
+        final ValueMapperWithKey mapper,
+        final Materialized> materialized);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -626,13 +628,14 @@  KTable mapValues(final ValueMapperWithKey   the value type of the result {@code KTable}
+     * @param    the value type of the result {@code KTable}
      *
      * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
      */
-     KTable mapValues(final ValueMapperWithKey mapper,
-                                 final Named named,
-                                 final Materialized> materialized);
+     KTable mapValues(
+        final ValueMapperWithKey mapper,
+        final Named named,
+        final Materialized> materialized);
 
     /**
      * Convert this changelog stream to a {@link KStream}.
@@ -678,10 +681,11 @@  KTable mapValues(final ValueMapperWithKey the new key type of the result stream
+     * @param  the new key type of the result stream
      * @return a {@link KStream} that contains the same records as this {@code KTable}
      */
-     KStream toStream(final KeyValueMapper mapper);
+     KStream toStream(
+        final KeyValueMapper mapper);
 
     /**
      * Convert this changelog stream to a {@link KStream} using the given {@link KeyValueMapper} to select the new key.
@@ -706,11 +710,12 @@  KTable mapValues(final ValueMapperWithKey the new key type of the result stream
+     * @param  the new key type of the result stream
      * @return a {@link KStream} that contains the same records as this {@code KTable}
      */
-     KStream toStream(final KeyValueMapper mapper,
-                                 final Named named);
+     KStream toStream(
+        final KeyValueMapper mapper,
+        final Named named);
 
     /**
      * Suppress some updates from this changelog stream, determined by the supplied {@link Suppressed} configuration.
@@ -787,13 +792,14 @@  KStream toStream(final KeyValueMapper                the value type of the result table
+     * @param                 the value type of the result table
      * @return a {@code KTable} that contains records with unmodified key and new values (possibly of different type)
      * @see #mapValues(ValueMapper)
      * @see #mapValues(ValueMapperWithKey)
      */
-     KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier,
-                                       final String... stateStoreNames);
+     KTable transformValues(
+        final ValueTransformerWithKeySupplier transformerSupplier,
+        final String... stateStoreNames);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -861,14 +867,15 @@  KTable transformValues(final ValueTransformerWithKeySupplier                the value type of the result table
+     * @param                 the value type of the result table
      * @return a {@code KTable} that contains records with unmodified key and new values (possibly of different type)
      * @see #mapValues(ValueMapper)
      * @see #mapValues(ValueMapperWithKey)
      */
-     KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier,
-                                       final Named named,
-                                       final String... stateStoreNames);
+     KTable transformValues(
+        final ValueTransformerWithKeySupplier transformerSupplier,
+        final Named named,
+        final String... stateStoreNames);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -940,14 +947,15 @@  KTable transformValues(final ValueTransformerWithKeySupplier                the value type of the result table
+     * @param                 the value type of the result table
      * @return a {@code KTable} that contains records with unmodified key and new values (possibly of different type)
      * @see #mapValues(ValueMapper)
      * @see #mapValues(ValueMapperWithKey)
      */
-     KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier,
-                                       final Materialized> materialized,
-                                       final String... stateStoreNames);
+     KTable transformValues(
+        final ValueTransformerWithKeySupplier transformerSupplier,
+        final Materialized> materialized,
+        final String... stateStoreNames);
 
     /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
@@ -1020,15 +1028,16 @@  KTable transformValues(final ValueTransformerWithKeySupplier                the value type of the result table
+     * @param                 the value type of the result table
      * @return a {@code KTable} that contains records with unmodified key and new values (possibly of different type)
      * @see #mapValues(ValueMapper)
      * @see #mapValues(ValueMapperWithKey)
      */
-     KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier,
-                                       final Materialized> materialized,
-                                       final Named named,
-                                       final String... stateStoreNames);
+     KTable transformValues(
+        final ValueTransformerWithKeySupplier transformerSupplier,
+        final Materialized> materialized,
+        final Named named,
+        final String... stateStoreNames);
 
     /**
      * Re-groups the records of this {@code KTable} using the provided {@link KeyValueMapper} and default serializers
@@ -1056,11 +1065,12 @@  KTable transformValues(final ValueTransformerWithKeySupplier     the key type of the result {@link KGroupedTable}
-     * @param      the value type of the result {@link KGroupedTable}
+     * @param      the key type of the result {@link KGroupedTable}
+     * @param      the value type of the result {@link KGroupedTable}
      * @return a {@link KGroupedTable} that contains the re-grouped records of the original {@code KTable}
      */
-     KGroupedTable groupBy(final KeyValueMapper> selector);
+     KGroupedTable groupBy(
+        final KeyValueMapper> selector);
 
     /**
      * Re-groups the records of this {@code KTable} using the provided {@link KeyValueMapper}
@@ -1088,12 +1098,13 @@  KTable transformValues(final ValueTransformerWithKeySupplier          the key type of the result {@link KGroupedTable}
-     * @param           the value type of the result {@link KGroupedTable}
+     * @param           the key type of the result {@link KGroupedTable}
+     * @param           the value type of the result {@link KGroupedTable}
      * @return a {@link KGroupedTable} that contains the re-grouped records of the original {@code KTable}
      */
-     KGroupedTable groupBy(final KeyValueMapper> selector,
-                                           final Grouped grouped);
+     KGroupedTable groupBy(
+        final KeyValueMapper> selector,
+        final Grouped grouped);
 
     /**
      * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join,
@@ -1160,15 +1171,15 @@  KGroupedTable groupBy(final KeyValueMapper   the value type of the other {@code KTable}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the other {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key
      * @see #leftJoin(KTable, ValueJoiner)
      * @see #outerJoin(KTable, ValueJoiner)
      */
-     KTable join(final KTable other,
-                                final ValueJoiner joiner);
+     KTable join(final KTable other,
+                                    final ValueJoiner joiner);
 
     /**
      * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join,
@@ -1236,16 +1247,16 @@  KTable join(final KTable other,
      * @param other  the other {@code KTable} to be joined with this {@code KTable}
      * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
      * @param named  a {@link Named} config used to name the processor in the topology
-     * @param    the value type of the other {@code KTable}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the other {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key
      * @see #leftJoin(KTable, ValueJoiner)
      * @see #outerJoin(KTable, ValueJoiner)
      */
-     KTable join(final KTable other,
-                                final ValueJoiner joiner,
-                                final Named named);
+     KTable join(final KTable other,
+                                    final ValueJoiner joiner,
+                                    final Named named);
 
     /**
      * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join,
@@ -1315,16 +1326,16 @@  KTable join(final KTable other,
      * @param joiner        a {@link ValueJoiner} that computes the join result for a pair of matching records
      * @param materialized  an instance of {@link Materialized} used to describe how the state store should be materialized.
      *                      Cannot be {@code null}
-     * @param           the value type of the other {@code KTable}
-     * @param           the value type of the result {@code KTable}
+     * @param           the value type of the other {@code KTable}
+     * @param           the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key
      * @see #leftJoin(KTable, ValueJoiner, Materialized)
      * @see #outerJoin(KTable, ValueJoiner, Materialized)
      */
-     KTable join(final KTable other,
-                                final ValueJoiner joiner,
-                                final Materialized> materialized);
+     KTable join(final KTable other,
+                                    final ValueJoiner joiner,
+                                    final Materialized> materialized);
 
     /**
      * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join,
@@ -1395,17 +1406,17 @@  KTable join(final KTable other,
      * @param named         a {@link Named} config used to name the processor in the topology
      * @param materialized  an instance of {@link Materialized} used to describe how the state store should be materialized.
      *                      Cannot be {@code null}
-     * @param           the value type of the other {@code KTable}
-     * @param           the value type of the result {@code KTable}
+     * @param           the value type of the other {@code KTable}
+     * @param           the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key
      * @see #leftJoin(KTable, ValueJoiner, Materialized)
      * @see #outerJoin(KTable, ValueJoiner, Materialized)
      */
-     KTable join(final KTable other,
-                                final ValueJoiner joiner,
-                                final Named named,
-                                final Materialized> materialized);
+     KTable join(final KTable other,
+                                    final ValueJoiner joiner,
+                                    final Named named,
+                                    final Materialized> materialized);
 
     /**
      * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
@@ -1478,16 +1489,16 @@  KTable join(final KTable other,
      *
      * @param other  the other {@code KTable} to be joined with this {@code KTable}
      * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
-     * @param    the value type of the other {@code KTable}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the other {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * left {@code KTable}
      * @see #join(KTable, ValueJoiner)
      * @see #outerJoin(KTable, ValueJoiner)
      */
-     KTable leftJoin(final KTable other,
-                                    final ValueJoiner joiner);
+     KTable leftJoin(final KTable other,
+                                        final ValueJoiner joiner);
 
     /**
      * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
@@ -1561,17 +1572,17 @@  KTable leftJoin(final KTable other,
      * @param other  the other {@code KTable} to be joined with this {@code KTable}
      * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
      * @param named  a {@link Named} config used to name the processor in the topology
-     * @param    the value type of the other {@code KTable}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the other {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * left {@code KTable}
      * @see #join(KTable, ValueJoiner)
      * @see #outerJoin(KTable, ValueJoiner)
      */
-     KTable leftJoin(final KTable other,
-                                    final ValueJoiner joiner,
-                                    final Named named);
+     KTable leftJoin(final KTable other,
+                                        final ValueJoiner joiner,
+                                        final Named named);
 
     /**
      * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
@@ -1647,17 +1658,17 @@  KTable leftJoin(final KTable other,
      * @param joiner        a {@link ValueJoiner} that computes the join result for a pair of matching records
      * @param materialized  an instance of {@link Materialized} used to describe how the state store should be materialized.
      *                      Cannot be {@code null}
-     * @param           the value type of the other {@code KTable}
-     * @param           the value type of the result {@code KTable}
+     * @param           the value type of the other {@code KTable}
+     * @param           the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * left {@code KTable}
      * @see #join(KTable, ValueJoiner, Materialized)
      * @see #outerJoin(KTable, ValueJoiner, Materialized)
      */
-     KTable leftJoin(final KTable other,
-                                    final ValueJoiner joiner,
-                                    final Materialized> materialized);
+     KTable leftJoin(final KTable other,
+                                        final ValueJoiner joiner,
+                                        final Materialized> materialized);
 
     /**
      * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
@@ -1734,18 +1745,18 @@  KTable leftJoin(final KTable other,
      * @param named         a {@link Named} config used to name the processor in the topology
      * @param materialized  an instance of {@link Materialized} used to describe how the state store should be materialized.
      *                      Cannot be {@code null}
-     * @param           the value type of the other {@code KTable}
-     * @param           the value type of the result {@code KTable}
+     * @param           the value type of the other {@code KTable}
+     * @param           the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * left {@code KTable}
      * @see #join(KTable, ValueJoiner, Materialized)
      * @see #outerJoin(KTable, ValueJoiner, Materialized)
      */
-     KTable leftJoin(final KTable other,
-                                    final ValueJoiner joiner,
-                                    final Named named,
-                                    final Materialized> materialized);
+     KTable leftJoin(final KTable other,
+                                        final ValueJoiner joiner,
+                                        final Named named,
+                                        final Materialized> materialized);
 
     /**
      * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
@@ -1817,16 +1828,16 @@  KTable leftJoin(final KTable other,
      *
      * @param other  the other {@code KTable} to be joined with this {@code KTable}
      * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
-     * @param    the value type of the other {@code KTable}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the other {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * both {@code KTable}s
      * @see #join(KTable, ValueJoiner)
      * @see #leftJoin(KTable, ValueJoiner)
      */
-     KTable outerJoin(final KTable other,
-                                     final ValueJoiner joiner);
+     KTable outerJoin(final KTable other,
+                                         final ValueJoiner joiner);
 
 
     /**
@@ -1900,17 +1911,17 @@  KTable outerJoin(final KTable other,
      * @param other  the other {@code KTable} to be joined with this {@code KTable}
      * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
      * @param named  a {@link Named} config used to name the processor in the topology
-     * @param    the value type of the other {@code KTable}
-     * @param    the value type of the result {@code KTable}
+     * @param    the value type of the other {@code KTable}
+     * @param    the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * both {@code KTable}s
      * @see #join(KTable, ValueJoiner)
      * @see #leftJoin(KTable, ValueJoiner)
      */
-     KTable outerJoin(final KTable other,
-                                     final ValueJoiner joiner,
-                                     final Named named);
+     KTable outerJoin(final KTable other,
+                                         final ValueJoiner joiner,
+                                         final Named named);
 
     /**
      * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
@@ -1985,17 +1996,17 @@  KTable outerJoin(final KTable other,
      * @param joiner        a {@link ValueJoiner} that computes the join result for a pair of matching records
      * @param materialized  an instance of {@link Materialized} used to describe how the state store should be materialized.
      *                      Cannot be {@code null}
-     * @param           the value type of the other {@code KTable}
-     * @param           the value type of the result {@code KTable}
+     * @param           the value type of the other {@code KTable}
+     * @param           the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * both {@code KTable}s
      * @see #join(KTable, ValueJoiner)
      * @see #leftJoin(KTable, ValueJoiner)
      */
-     KTable outerJoin(final KTable other,
-                                     final ValueJoiner joiner,
-                                     final Materialized> materialized);
+     KTable outerJoin(final KTable other,
+                                         final ValueJoiner joiner,
+                                         final Materialized> materialized);
 
 
     /**
@@ -2072,100 +2083,122 @@  KTable outerJoin(final KTable other,
      * @param named         a {@link Named} config used to name the processor in the topology
      * @param materialized  an instance of {@link Materialized} used to describe how the state store should be materialized.
      *                      Cannot be {@code null}
-     * @param           the value type of the other {@code KTable}
-     * @param           the value type of the result {@code KTable}
+     * @param           the value type of the other {@code KTable}
+     * @param           the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
      * both {@code KTable}s
      * @see #join(KTable, ValueJoiner)
      * @see #leftJoin(KTable, ValueJoiner)
      */
-     KTable outerJoin(final KTable other,
-                                     final ValueJoiner joiner,
-                                     final Named named,
-                                     final Materialized> materialized);
+     KTable outerJoin(final KTable other,
+                                         final ValueJoiner joiner,
+                                         final Named named,
+                                         final Materialized> materialized);
 
     /**
-     * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join.
-     * 

- * This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}. - * - * @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO. - * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the - * result is null, the update is ignored as invalid. - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the value type of the result {@code KTable} - * @param the key type of the other {@code KTable} - * @param the value type of the other {@code KTable} + * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner + * join. + *

+ * This is a foreign key join, where the joining key is determined by the {@code + * foreignKeyExtractor}. + * + * @param other the other {@code KTable} to be joined with this {@code KTable}. + * Keyed by KO. + * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's + * value (V). If the result is null, the update is ignored as + * invalid. + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of + * matching records + * @param the key type of the other {@code KTable} + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} */ - KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner); + KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner); /** - * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join. - *

- * This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}. - * - * @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO. - * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the - * result is null, the update is ignored as invalid. - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner + * join. + *

+ * This is a foreign key join, where the joining key is determined by the {@code + * foreignKeyExtractor}. + * + * @param other the other {@code KTable} to be joined with this {@code KTable}. + * Keyed by KO. + * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's + * value (V). If the result is null, the update is ignored as + * invalid. + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of + * matching records * @param named a {@link Named} config used to name the processor in the topology - * @param the value type of the result {@code KTable} - * @param the key type of the other {@code KTable} - * @param the value type of the other {@code KTable} + * @param the key type of the other {@code KTable} + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} */ - KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named); + KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named); /** - * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join. - *

- * This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}. - * - * @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO. - * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the - * result is null, the update is ignored as invalid. - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable} - * should be materialized. Cannot be {@code null} - * @param the value type of the result {@code KTable} - * @param the key type of the other {@code KTable} - * @param the value type of the other {@code KTable} + * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner + * join. + *

+ * This is a foreign key join, where the joining key is determined by the {@code + * foreignKeyExtractor}. + * + * @param other the other {@code KTable} to be joined with this {@code KTable}. + * Keyed by KO. + * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's + * value (V). If the result is null, the update is ignored as + * invalid. + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of + * matching records + * @param materialized a {@link Materialized} that describes how the {@link StateStore} + * for the resulting {@code KTable} should be materialized. Cannot be + * {@code null} + * @param the key type of the other {@code KTable} + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} */ - KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Materialized> materialized); + KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Materialized> materialized); /** - * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join. - *

- * This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}. - * - * @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO. - * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the - * result is null, the update is ignored as invalid. - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner + * join. + *

+ * This is a foreign key join, where the joining key is determined by the {@code + * foreignKeyExtractor}. + * + * @param other the other {@code KTable} to be joined with this {@code KTable}. + * Keyed by KO. + * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's + * value (V). If the result is null, the update is ignored as + * invalid. + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of + * matching records * @param named a {@link Named} config used to name the processor in the topology - * @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable} - * should be materialized. Cannot be {@code null} - * @param the value type of the result {@code KTable} - * @param the key type of the other {@code KTable} - * @param the value type of the other {@code KTable} + * @param materialized a {@link Materialized} that describes how the {@link StateStore} + * for the resulting {@code KTable} should be materialized. Cannot be + * {@code null} + * @param the key type of the other {@code KTable} + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} */ - KTable join(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named, - final Materialized> materialized); + KTable join(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named, + final Materialized> materialized); /** * Join records of this {@code KTable} with another {@code KTable} using non-windowed left join. @@ -2176,14 +2209,14 @@ KTable join(final KTable other, * @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the * result is null, the update is ignored as invalid. * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the value type of the result {@code KTable} - * @param the key type of the other {@code KTable} - * @param the value type of the other {@code KTable} + * @param the key type of the other {@code KTable} + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains only those records that satisfy the given predicate */ - KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner); + KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner); /** * Join records of this {@code KTable} with another {@code KTable} using non-windowed left join. @@ -2195,15 +2228,15 @@ KTable leftJoin(final KTable other, * result is null, the update is ignored as invalid. * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param named a {@link Named} config used to name the processor in the topology - * @param the value type of the result {@code KTable} * @param the key type of the other {@code KTable} * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} */ - KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named); + KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named); /** * Join records of this {@code KTable} with another {@code KTable} using non-windowed left join. @@ -2216,15 +2249,15 @@ KTable leftJoin(final KTable other, * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable} * should be materialized. Cannot be {@code null} - * @param the value type of the result {@code KTable} - * @param the key type of the other {@code KTable} - * @param the value type of the other {@code KTable} + * @param the key type of the other {@code KTable} + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} */ - KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Materialized> materialized); + KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Materialized> materialized); /** * Join records of this {@code KTable} with another {@code KTable} using non-windowed left join. @@ -2238,16 +2271,16 @@ KTable leftJoin(final KTable other, * @param named a {@link Named} config used to name the processor in the topology * @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable} * should be materialized. Cannot be {@code null} - * @param the value type of the result {@code KTable} - * @param the key type of the other {@code KTable} - * @param the value type of the other {@code KTable} + * @param the key type of the other {@code KTable} + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} */ - KTable leftJoin(final KTable other, - final Function foreignKeyExtractor, - final ValueJoiner joiner, - final Named named, - final Materialized> materialized); + KTable leftJoin(final KTable other, + final Function foreignKeyExtractor, + final ValueJoiner joiner, + final Named named, + final Materialized> materialized); /** * Get the name of the local state store used that can be used to query this {@code KTable}. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java index 0e573750d2a3b..62a409544e2bd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java @@ -23,9 +23,9 @@ * record-pair of a {@link KStream}-{@link KStream}, {@link KStream}-{@link KTable}, or {@link KTable}-{@link KTable} * join. * - * @param first value type - * @param second value type - * @param joined value type + * @param first value type + * @param second value type + * @param joined value type * @see KStream#join(KStream, ValueJoiner, JoinWindows) * @see KStream#join(KStream, ValueJoiner, JoinWindows, StreamJoined) * @see KStream#leftJoin(KStream, ValueJoiner, JoinWindows) @@ -40,7 +40,7 @@ * @see KTable#leftJoin(KTable, ValueJoiner) * @see KTable#outerJoin(KTable, ValueJoiner) */ -public interface ValueJoiner { +public interface ValueJoiner { /** * Return a joined value consisting of {@code value1} and {@code value2}. @@ -49,5 +49,5 @@ public interface ValueJoiner { * @param value2 the second value for joining * @return the joined value */ - VR apply(final V1 value1, final V2 value2); + VOut apply(final V value1, final V1 value2); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoinerWithKey.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoinerWithKey.java index 57f76c88eb1a4..998784c912d99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoinerWithKey.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoinerWithKey.java @@ -25,10 +25,10 @@ * record-pair of a {@link KStream}-{@link KStream}, {@link KStream}-{@link KTable}, or {@link KTable}-{@link KTable} * join. * - * @param key value type - * @param first value type - * @param second value type - * @param joined value type + * @param key value type + * @param first value type + * @param second value type + * @param joined value type * @see KStream#join(KStream, ValueJoinerWithKey, JoinWindows) * @see KStream#join(KStream, ValueJoinerWithKey, JoinWindows, StreamJoined) * @see KStream#leftJoin(KStream, ValueJoinerWithKey, JoinWindows) @@ -44,7 +44,7 @@ * @see KStream#leftJoin(GlobalKTable, KeyValueMapper, ValueJoinerWithKey) * @see KStream#leftJoin(GlobalKTable, KeyValueMapper, ValueJoinerWithKey, Named) */ -public interface ValueJoinerWithKey { +public interface ValueJoinerWithKey { /** * Return a joined value consisting of {@code readOnlyKey}, {@code value1} and {@code value2}. @@ -54,5 +54,5 @@ public interface ValueJoinerWithKey { * @param value2 the second value for joining * @return the joined value */ - VR apply(final K1 readOnlyKey, final V1 value1, final V2 value2); + VOut apply(final K readOnlyKey, final V value1, final V1 value2); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java index c5ab2a8bfc202..4bb61fedcbd5b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java @@ -143,7 +143,7 @@ public Set> stores() { }; } - static ValueJoinerWithKey toValueJoinerWithKey(final ValueJoiner valueJoiner) { + static ValueJoinerWithKey toValueJoinerWithKey(final ValueJoiner valueJoiner) { Objects.requireNonNull(valueJoiner, "joiner can't be null"); return (readOnlyKey, value1, value2) -> valueJoiner.apply(value1, value2); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java index f9d43a663743e..70f370f3d794c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java @@ -21,16 +21,16 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -class KStreamGlobalKTableJoin implements ProcessorSupplier { +class KStreamGlobalKTableJoin implements ProcessorSupplier { - private final KTableValueGetterSupplier valueGetterSupplier; - private final ValueJoinerWithKey joiner; - private final KeyValueMapper mapper; + private final KTableValueGetterSupplier valueGetterSupplier; + private final ValueJoinerWithKey joiner; + private final KeyValueMapper mapper; private final boolean leftJoin; - KStreamGlobalKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, - final ValueJoinerWithKey joiner, - final KeyValueMapper mapper, + KStreamGlobalKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, + final ValueJoinerWithKey joiner, + final KeyValueMapper mapper, final boolean leftJoin) { this.valueGetterSupplier = valueGetterSupplier; this.joiner = joiner; @@ -39,7 +39,7 @@ class KStreamGlobalKTableJoin implements ProcessorSupplier } @Override - public Processor get() { + public Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), mapper, joiner, leftJoin); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 16c8f05e54b98..c7d6b615830ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -205,12 +205,12 @@ public KStream filterNot(final Predicate predicate, } @Override - public KStream selectKey(final KeyValueMapper mapper) { + public KStream selectKey(final KeyValueMapper mapper) { return selectKey(mapper, NamedInternal.empty()); } @Override - public KStream selectKey(final KeyValueMapper mapper, + public KStream selectKey(final KeyValueMapper mapper, final Named named) { Objects.requireNonNull(mapper, "mapper can't be null"); Objects.requireNonNull(named, "named can't be null"); @@ -231,10 +231,10 @@ public KStream selectKey(final KeyValueMapper ProcessorGraphNode internalSelectKey(final KeyValueMapper mapper, - final NamedInternal named) { + private ProcessorGraphNode internalSelectKey(final KeyValueMapper mapper, + final NamedInternal named) { final String name = named.orElseGenerateWithPrefix(builder, KEY_SELECT_NAME); - final KStreamMap kStreamMap = + final KStreamMap kStreamMap = new KStreamMap<>((key, value) -> new KeyValue<>(mapper.apply(key, value), value)); final ProcessorParameters processorParameters = new ProcessorParameters<>(kStreamMap, name); @@ -242,12 +242,12 @@ private ProcessorGraphNode internalSelectKey(final KeyValueMapper KStream map(final KeyValueMapper> mapper) { + public KStream map(final KeyValueMapper> mapper) { return map(mapper, NamedInternal.empty()); } @Override - public KStream map(final KeyValueMapper> mapper, + public KStream map(final KeyValueMapper> mapper, final Named named) { Objects.requireNonNull(mapper, "mapper can't be null"); Objects.requireNonNull(named, "named can't be null"); @@ -273,24 +273,24 @@ public KStream map(final KeyValueMapper KStream mapValues(final ValueMapper valueMapper) { + public KStream mapValues(final ValueMapper valueMapper) { return mapValues(withKey(valueMapper)); } @Override - public KStream mapValues(final ValueMapper mapper, + public KStream mapValues(final ValueMapper mapper, final Named named) { return mapValues(withKey(mapper), named); } @Override - public KStream mapValues(final ValueMapperWithKey valueMapperWithKey) { + public KStream mapValues(final ValueMapperWithKey valueMapperWithKey) { return mapValues(valueMapperWithKey, NamedInternal.empty()); } @Override - public KStream mapValues(final ValueMapperWithKey valueMapperWithKey, - final Named named) { + public KStream mapValues(final ValueMapperWithKey valueMapperWithKey, + final Named named) { Objects.requireNonNull(valueMapperWithKey, "valueMapperWithKey can't be null"); Objects.requireNonNull(named, "named can't be null"); @@ -315,12 +315,12 @@ public KStream mapValues(final ValueMapperWithKey KStream flatMap(final KeyValueMapper>> mapper) { + public KStream flatMap(final KeyValueMapper>> mapper) { return flatMap(mapper, NamedInternal.empty()); } @Override - public KStream flatMap(final KeyValueMapper>> mapper, + public KStream flatMap(final KeyValueMapper>> mapper, final Named named) { Objects.requireNonNull(mapper, "mapper can't be null"); Objects.requireNonNull(named, "named can't be null"); @@ -339,24 +339,24 @@ public KStream flatMap(final KeyValueMapper KStream flatMapValues(final ValueMapper> mapper) { + public KStream flatMapValues(final ValueMapper> mapper) { return flatMapValues(withKey(mapper)); } @Override - public KStream flatMapValues(final ValueMapper> mapper, + public KStream flatMapValues(final ValueMapper> mapper, final Named named) { return flatMapValues(withKey(mapper), named); } @Override - public KStream flatMapValues(final ValueMapperWithKey> mapper) { + public KStream flatMapValues(final ValueMapperWithKey> mapper) { return flatMapValues(mapper, NamedInternal.empty()); } @Override - public KStream flatMapValues(final ValueMapperWithKey> valueMapper, - final Named named) { + public KStream flatMapValues(final ValueMapperWithKey> valueMapper, + final Named named) { Objects.requireNonNull(valueMapper, "valueMapper can't be null"); Objects.requireNonNull(named, "named can't be null"); @@ -774,17 +774,17 @@ public KTable toTable(final Named named, } @Override - public KGroupedStream groupBy(final KeyValueMapper keySelector) { + public KGroupedStream groupBy(final KeyValueMapper keySelector) { return groupBy(keySelector, Grouped.with(null, valueSerde)); } @Override - public KGroupedStream groupBy(final KeyValueMapper keySelector, - final Grouped grouped) { + public KGroupedStream groupBy(final KeyValueMapper keySelector, + final Grouped grouped) { Objects.requireNonNull(keySelector, "keySelector can't be null"); Objects.requireNonNull(grouped, "grouped can't be null"); - final GroupedInternal groupedInternal = new GroupedInternal<>(grouped); + final GroupedInternal groupedInternal = new GroupedInternal<>(grouped); final ProcessorGraphNode selectKeyMapNode = internalSelectKey(keySelector, new NamedInternal(groupedInternal.name())); selectKeyMapNode.keyChangingOperation(true); @@ -820,33 +820,33 @@ public KGroupedStream groupByKey(final Grouped grouped) { } @Override - public KStream join(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows) { + public KStream join(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows) { return join(otherStream, toValueJoinerWithKey(joiner), windows); } @Override - public KStream join(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows) { + public KStream join(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows) { return join(otherStream, joiner, windows, StreamJoined.with(null, null, null)); } @Override - public KStream join(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream join(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { return join(otherStream, toValueJoinerWithKey(joiner), windows, streamJoined); } @Override - public KStream join(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream join(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { return doJoin( otherStream, @@ -857,24 +857,24 @@ public KStream join(final KStream otherStream, } @Override - public KStream leftJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows) { + public KStream leftJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows) { return leftJoin(otherStream, toValueJoinerWithKey(joiner), windows); } @Override - public KStream leftJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows) { + public KStream leftJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows) { return leftJoin(otherStream, joiner, windows, StreamJoined.with(null, null, null)); } @Override - public KStream leftJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream leftJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { return doJoin( otherStream, toValueJoinerWithKey(joiner), @@ -884,10 +884,10 @@ public KStream leftJoin(final KStream otherStream, } @Override - public KStream leftJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream leftJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { return doJoin( otherStream, joiner, @@ -897,51 +897,51 @@ public KStream leftJoin(final KStream otherStream, } @Override - public KStream outerJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows) { + public KStream outerJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows) { return outerJoin(otherStream, toValueJoinerWithKey(joiner), windows); } @Override - public KStream outerJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows) { + public KStream outerJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows) { return outerJoin(otherStream, joiner, windows, StreamJoined.with(null, null, null)); } @Override - public KStream outerJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream outerJoin(final KStream otherStream, + final ValueJoiner joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { return outerJoin(otherStream, toValueJoinerWithKey(joiner), windows, streamJoined); } @Override - public KStream outerJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream outerJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { return doJoin(otherStream, joiner, windows, streamJoined, new KStreamImplJoin(builder, true, true)); } - private KStream doJoin(final KStream otherStream, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined, - final KStreamImplJoin join) { + private KStream doJoin(final KStream otherStream, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined, + final KStreamImplJoin join) { Objects.requireNonNull(otherStream, "otherStream can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(windows, "windows can't be null"); Objects.requireNonNull(streamJoined, "streamJoined can't be null"); KStreamImpl joinThis = this; - KStreamImpl joinOther = (KStreamImpl) otherStream; + KStreamImpl joinOther = (KStreamImpl) otherStream; - final StreamJoinedInternal streamJoinedInternal = new StreamJoinedInternal<>(streamJoined); + final StreamJoinedInternal streamJoinedInternal = new StreamJoinedInternal<>(streamJoined); final NamedInternal name = new NamedInternal(streamJoinedInternal.name()); if (joinThis.repartitionRequired) { final String joinThisName = joinThis.name; @@ -1053,21 +1053,21 @@ static > String createRepartition } @Override - public KStream join(final KTable table, - final ValueJoiner joiner) { + public KStream join(final KTable table, + final ValueJoiner joiner) { return join(table, toValueJoinerWithKey(joiner)); } @Override - public KStream join(final KTable table, - final ValueJoinerWithKey joiner) { + public KStream join(final KTable table, + final ValueJoinerWithKey joiner) { return join(table, joiner, Joined.with(null, null, null)); } @Override - public KStream join(final KTable table, - final ValueJoiner joiner, - final Joined joined) { + public KStream join(final KTable table, + final ValueJoiner joiner, + final Joined joined) { Objects.requireNonNull(table, "table can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(joined, "joined can't be null"); @@ -1075,14 +1075,14 @@ public KStream join(final KTable table, } @Override - public KStream join(final KTable table, - final ValueJoinerWithKey joiner, - final Joined joined) { + public KStream join(final KTable table, + final ValueJoinerWithKey joiner, + final Joined joined) { Objects.requireNonNull(table, "table can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(joined, "joined can't be null"); - final JoinedInternal joinedInternal = new JoinedInternal<>(joined); + final JoinedInternal joinedInternal = new JoinedInternal<>(joined); final String name = joinedInternal.name(); if (repartitionRequired) { @@ -1098,19 +1098,19 @@ public KStream join(final KTable table, } @Override - public KStream leftJoin(final KTable table, final ValueJoiner joiner) { + public KStream leftJoin(final KTable table, final ValueJoiner joiner) { return leftJoin(table, toValueJoinerWithKey(joiner)); } @Override - public KStream leftJoin(final KTable table, final ValueJoinerWithKey joiner) { + public KStream leftJoin(final KTable table, final ValueJoinerWithKey joiner) { return leftJoin(table, joiner, Joined.with(null, null, null)); } @Override - public KStream leftJoin(final KTable table, - final ValueJoiner joiner, - final Joined joined) { + public KStream leftJoin(final KTable table, + final ValueJoiner joiner, + final Joined joined) { Objects.requireNonNull(table, "table can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(joined, "joined can't be null"); @@ -1119,13 +1119,13 @@ public KStream leftJoin(final KTable table, } @Override - public KStream leftJoin(final KTable table, - final ValueJoinerWithKey joiner, - final Joined joined) { + public KStream leftJoin(final KTable table, + final ValueJoinerWithKey joiner, + final Joined joined) { Objects.requireNonNull(table, "table can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); Objects.requireNonNull(joined, "joined can't be null"); - final JoinedInternal joinedInternal = new JoinedInternal<>(joined); + final JoinedInternal joinedInternal = new JoinedInternal<>(joined); final String name = joinedInternal.name(); if (repartitionRequired) { @@ -1141,70 +1141,70 @@ public KStream leftJoin(final KTable table, } @Override - public KStream join(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoiner joiner) { + public KStream join(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoiner joiner) { return join(globalTable, keySelector, toValueJoinerWithKey(joiner)); } @Override - public KStream join(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey joiner) { + public KStream join(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey joiner) { return globalTableJoin(globalTable, keySelector, joiner, false, NamedInternal.empty()); } @Override - public KStream join(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoiner joiner, - final Named named) { + public KStream join(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoiner joiner, + final Named named) { return join(globalTable, keySelector, toValueJoinerWithKey(joiner), named); } @Override - public KStream join(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey joiner, - final Named named) { + public KStream join(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey joiner, + final Named named) { return globalTableJoin(globalTable, keySelector, joiner, false, named); } @Override - public KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoiner joiner) { + public KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoiner joiner) { return leftJoin(globalTable, keySelector, toValueJoinerWithKey(joiner)); } @Override - public KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey joiner) { + public KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey joiner) { return globalTableJoin(globalTable, keySelector, joiner, true, NamedInternal.empty()); } @Override - public KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoiner joiner, - final Named named) { + public KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoiner joiner, + final Named named) { return leftJoin(globalTable, keySelector, toValueJoinerWithKey(joiner), named); } @Override - public KStream leftJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey joiner, - final Named named) { + public KStream leftJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey joiner, + final Named named) { return globalTableJoin(globalTable, keySelector, joiner, true, named); } - private KStream globalTableJoin(final GlobalKTable globalTable, - final KeyValueMapper keySelector, - final ValueJoinerWithKey joiner, - final boolean leftJoin, - final Named named) { + private KStream globalTableJoin(final GlobalKTable globalTable, + final KeyValueMapper keySelector, + final ValueJoinerWithKey joiner, + final boolean leftJoin, + final Named named) { Objects.requireNonNull(globalTable, "globalTable can't be null"); Objects.requireNonNull(keySelector, "keySelector can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); @@ -1214,12 +1214,12 @@ private KStream globalTableJoin(final GlobalKTable g ((GlobalKTableImpl) globalTable).valueGetterSupplier(); final String name = new NamedInternal(named) .orElseGenerateWithPrefix(builder, LEFTJOIN_NAME); - final ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( + final ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( valueGetterSupplier, joiner, keySelector, leftJoin); - final ProcessorParameters processorParameters = new ProcessorParameters<>( + final ProcessorParameters processorParameters = new ProcessorParameters<>( processorSupplier, name); final StreamTableJoinNode streamTableJoinNode = new StreamTableJoinNode<>( name, processorParameters, new String[] {}, null); @@ -1238,21 +1238,21 @@ private KStream globalTableJoin(final GlobalKTable g } @SuppressWarnings("unchecked") - private KStream doStreamTableJoin(final KTable table, - final ValueJoinerWithKey joiner, - final Joined joined, + private KStream doStreamTableJoin(final KTable table, + final ValueJoinerWithKey joiner, + final Joined joined, final boolean leftJoin) { Objects.requireNonNull(table, "table can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); - final Set allSourceNodes = ensureCopartitionWith(Collections.singleton((AbstractStream) table)); + final Set allSourceNodes = ensureCopartitionWith(Collections.singleton((AbstractStream) table)); - final JoinedInternal joinedInternal = new JoinedInternal<>(joined); + final JoinedInternal joinedInternal = new JoinedInternal<>(joined); final NamedInternal renamed = new NamedInternal(joinedInternal.name()); final String name = renamed.orElseGenerateWithPrefix(builder, leftJoin ? LEFTJOIN_NAME : JOIN_NAME); - final KStreamKTableJoin processorSupplier = new KStreamKTableJoin<>( - ((KTableImpl) table).valueGetterSupplier(), + final KStreamKTableJoin processorSupplier = new KStreamKTableJoin<>( + ((KTableImpl) table).valueGetterSupplier(), joiner, leftJoin); @@ -1261,7 +1261,7 @@ private KStream doStreamTableJoin(final KTable table, final StreamTableJoinNode streamTableJoinNode = new StreamTableJoinNode<>( name, processorParameters, - ((KTableImpl) table).valueGetterSupplier().storeNames(), + ((KTableImpl) table).valueGetterSupplier().storeNames(), this.name ); @@ -1279,7 +1279,7 @@ private KStream doStreamTableJoin(final KTable table, } @Override - public KStream transform(final TransformerSupplier> transformerSupplier, + public KStream transform(final TransformerSupplier> transformerSupplier, final String... stateStoreNames) { Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null"); final String name = builder.newProcessorName(TRANSFORM_NAME); @@ -1287,7 +1287,7 @@ public KStream transform(final TransformerSupplier KStream transform(final TransformerSupplier> transformerSupplier, + public KStream transform(final TransformerSupplier> transformerSupplier, final Named named, final String... stateStoreNames) { Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null"); @@ -1295,17 +1295,17 @@ public KStream transform(final TransformerSupplier KStream flatTransform(final TransformerSupplier>> transformerSupplier, - final String... stateStoreNames) { + public KStream flatTransform(final TransformerSupplier>> transformerSupplier, + final String... stateStoreNames) { Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null"); final String name = builder.newProcessorName(TRANSFORM_NAME); return flatTransform(transformerSupplier, Named.as(name), stateStoreNames); } @Override - public KStream flatTransform(final TransformerSupplier>> transformerSupplier, - final Named named, - final String... stateStoreNames) { + public KStream flatTransform(final TransformerSupplier>> transformerSupplier, + final Named named, + final String... stateStoreNames) { Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null"); Objects.requireNonNull(named, "named can't be null"); Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); @@ -1335,7 +1335,7 @@ public KStream flatTransform(final TransformerSupplier KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, + public KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); return doTransformValues( @@ -1345,7 +1345,7 @@ public KStream transformValues(final ValueTransformerSupplier KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, + public KStream transformValues(final ValueTransformerSupplier valueTransformerSupplier, final Named named, final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); @@ -1357,14 +1357,14 @@ public KStream transformValues(final ValueTransformerSupplier KStream transformValues(final ValueTransformerWithKeySupplier valueTransformerSupplier, + public KStream transformValues(final ValueTransformerWithKeySupplier valueTransformerSupplier, final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); return doTransformValues(valueTransformerSupplier, NamedInternal.empty(), stateStoreNames); } @Override - public KStream transformValues(final ValueTransformerWithKeySupplier valueTransformerSupplier, + public KStream transformValues(final ValueTransformerWithKeySupplier valueTransformerSupplier, final Named named, final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); @@ -1372,7 +1372,7 @@ public KStream transformValues(final ValueTransformerWithKeySupplier return doTransformValues(valueTransformerSupplier, new NamedInternal(named), stateStoreNames); } - private KStream doTransformValues(final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier, + private KStream doTransformValues(final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier, final NamedInternal named, final String... stateStoreNames) { Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); @@ -1402,8 +1402,8 @@ private KStream doTransformValues(final ValueTransformerWithKeySuppl } @Override - public KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, - final String... stateStoreNames) { + public KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, + final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); return doFlatTransformValues( toValueTransformerWithKeySupplier(valueTransformerSupplier), @@ -1412,9 +1412,9 @@ public KStream flatTransformValues(final ValueTransformerSupplier KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, - final Named named, - final String... stateStoreNames) { + public KStream flatTransformValues(final ValueTransformerSupplier> valueTransformerSupplier, + final Named named, + final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); return doFlatTransformValues( toValueTransformerWithKeySupplier(valueTransformerSupplier), @@ -1423,23 +1423,23 @@ public KStream flatTransformValues(final ValueTransformerSupplier KStream flatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerSupplier, - final String... stateStoreNames) { + public KStream flatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerSupplier, + final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); return doFlatTransformValues(valueTransformerSupplier, NamedInternal.empty(), stateStoreNames); } @Override - public KStream flatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerSupplier, - final Named named, - final String... stateStoreNames) { + public KStream flatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerSupplier, + final Named named, + final String... stateStoreNames) { Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null"); return doFlatTransformValues(valueTransformerSupplier, named, stateStoreNames); } - private KStream doFlatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerWithKeySupplier, - final Named named, - final String... stateStoreNames) { + private KStream doFlatTransformValues(final ValueTransformerWithKeySupplier> valueTransformerWithKeySupplier, + final Named named, + final String... stateStoreNames) { Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array"); for (final String stateStoreName : stateStoreNames) { Objects.requireNonNull(stateStoreName, "stateStoreNames can't contain `null` as store name"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java index 5af2e9c073c49..8d0a0e42a6c57 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java @@ -53,13 +53,13 @@ class KStreamImplJoin { this.rightOuter = rightOuter; } - public KStream join(final KStream lhs, - final KStream other, - final ValueJoinerWithKey joiner, - final JoinWindows windows, - final StreamJoined streamJoined) { + public KStream join(final KStream lhs, + final KStream other, + final ValueJoinerWithKey joiner, + final JoinWindows windows, + final StreamJoined streamJoined) { - final StreamJoinedInternal streamJoinedInternal = new StreamJoinedInternal<>(streamJoined); + final StreamJoinedInternal streamJoinedInternal = new StreamJoinedInternal<>(streamJoined); final NamedInternal renamed = new NamedInternal(streamJoinedInternal.name()); final String joinThisSuffix = rightOuter ? "-outer-this-join" : "-this-join"; final String joinOtherSuffix = leftOuter ? "-outer-other-join" : "-other-join"; @@ -81,8 +81,8 @@ public KStream join(final KStream lhs, final GraphNode thisGraphNode = ((AbstractStream) lhs).graphNode; final GraphNode otherGraphNode = ((AbstractStream) other).graphNode; - final StoreBuilder> thisWindowStore; - final StoreBuilder> otherWindowStore; + final StoreBuilder> thisWindowStore; + final StoreBuilder> otherWindowStore; final String userProvidedBaseStoreName = streamJoinedInternal.storeName(); final WindowBytesStoreSupplier thisStoreSupplier = streamJoinedInternal.thisStoreSupplier(); @@ -106,19 +106,19 @@ public KStream join(final KStream lhs, otherWindowStore = joinWindowStoreBuilderFromSupplier(otherStoreSupplier, streamJoinedInternal.keySerde(), streamJoinedInternal.otherValueSerde()); } - final KStreamJoinWindow thisWindowedStream = new KStreamJoinWindow<>(thisWindowStore.name()); + final KStreamJoinWindow thisWindowedStream = new KStreamJoinWindow<>(thisWindowStore.name()); - final ProcessorParameters thisWindowStreamProcessorParams = new ProcessorParameters<>(thisWindowedStream, thisWindowStreamProcessorName); - final ProcessorGraphNode thisWindowedStreamsNode = new ProcessorGraphNode<>(thisWindowStreamProcessorName, thisWindowStreamProcessorParams); + final ProcessorParameters thisWindowStreamProcessorParams = new ProcessorParameters<>(thisWindowedStream, thisWindowStreamProcessorName); + final ProcessorGraphNode thisWindowedStreamsNode = new ProcessorGraphNode<>(thisWindowStreamProcessorName, thisWindowStreamProcessorParams); builder.addGraphNode(thisGraphNode, thisWindowedStreamsNode); - final KStreamJoinWindow otherWindowedStream = new KStreamJoinWindow<>(otherWindowStore.name()); + final KStreamJoinWindow otherWindowedStream = new KStreamJoinWindow<>(otherWindowStore.name()); - final ProcessorParameters otherWindowStreamProcessorParams = new ProcessorParameters<>(otherWindowedStream, otherWindowStreamProcessorName); - final ProcessorGraphNode otherWindowedStreamsNode = new ProcessorGraphNode<>(otherWindowStreamProcessorName, otherWindowStreamProcessorParams); + final ProcessorParameters otherWindowStreamProcessorParams = new ProcessorParameters<>(otherWindowedStream, otherWindowStreamProcessorName); + final ProcessorGraphNode otherWindowedStreamsNode = new ProcessorGraphNode<>(otherWindowStreamProcessorName, otherWindowStreamProcessorParams); builder.addGraphNode(otherGraphNode, otherWindowedStreamsNode); - final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>( + final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>( otherWindowStore.name(), windows.beforeMs, windows.afterMs, @@ -126,7 +126,7 @@ public KStream join(final KStream lhs, leftOuter ); - final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>( + final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>( thisWindowStore.name(), windows.afterMs, windows.beforeMs, @@ -134,13 +134,13 @@ public KStream join(final KStream lhs, rightOuter ); - final PassThrough joinMerge = new PassThrough<>(); + final PassThrough joinMerge = new PassThrough<>(); - final StreamStreamJoinNode.StreamStreamJoinNodeBuilder joinBuilder = StreamStreamJoinNode.streamStreamJoinNodeBuilder(); + final StreamStreamJoinNode.StreamStreamJoinNodeBuilder joinBuilder = StreamStreamJoinNode.streamStreamJoinNodeBuilder(); - final ProcessorParameters joinThisProcessorParams = new ProcessorParameters<>(joinThis, joinThisName); - final ProcessorParameters joinOtherProcessorParams = new ProcessorParameters<>(joinOther, joinOtherName); - final ProcessorParameters joinMergeProcessorParams = new ProcessorParameters<>(joinMerge, joinMergeName); + final ProcessorParameters joinThisProcessorParams = new ProcessorParameters<>(joinThis, joinThisName); + final ProcessorParameters joinOtherProcessorParams = new ProcessorParameters<>(joinOther, joinOtherName); + final ProcessorParameters joinMergeProcessorParams = new ProcessorParameters<>(joinMerge, joinMergeName); joinBuilder.withJoinMergeProcessorParameters(joinMergeProcessorParams) .withJoinThisProcessorParameters(joinThisProcessorParams) @@ -156,8 +156,8 @@ public KStream join(final KStream lhs, builder.addGraphNode(Arrays.asList(thisGraphNode, otherGraphNode), joinGraphNode); - final Set allSourceNodes = new HashSet<>(((KStreamImpl) lhs).subTopologySourceNodes); - allSourceNodes.addAll(((KStreamImpl) other).subTopologySourceNodes); + final Set allSourceNodes = new HashSet<>(((KStreamImpl) lhs).subTopologySourceNodes); + allSourceNodes.addAll(((KStreamImpl) other).subTopologySourceNodes); // do not have serde for joined result; // also for key serde we do not inherit from either since we cannot tell if these two serdes are different diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index ee8c1d8f642fc..f6ca2e2d0c07b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -33,7 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class KStreamKStreamJoin implements ProcessorSupplier { +class KStreamKStreamJoin implements ProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); @@ -41,13 +41,13 @@ class KStreamKStreamJoin implements ProcessorSupplier joiner; + private final ValueJoinerWithKey joiner; private final boolean outer; KStreamKStreamJoin(final String otherWindowName, final long joinBeforeMs, final long joinAfterMs, - final ValueJoinerWithKey joiner, + final ValueJoinerWithKey joiner, final boolean outer) { this.otherWindowName = otherWindowName; this.joinBeforeMs = joinBeforeMs; @@ -57,13 +57,13 @@ class KStreamKStreamJoin implements ProcessorSupplier get() { + public Processor get() { return new KStreamKStreamJoinProcessor(); } - private class KStreamKStreamJoinProcessor extends ContextualProcessor { + private class KStreamKStreamJoinProcessor extends ContextualProcessor { - private WindowStore otherWindow; + private WindowStore otherWindow; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; @@ -77,7 +77,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(final Record record) { + public void process(final Record record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record // // we also ignore the record if value is null, because in a key-value data model a null-value indicates @@ -102,11 +102,11 @@ public void process(final Record record) { final long timeFrom = Math.max(0L, inputRecordTimestamp - joinBeforeMs); final long timeTo = Math.max(0L, inputRecordTimestamp + joinAfterMs); - try (final WindowStoreIterator iter = otherWindow + try (final WindowStoreIterator iter = otherWindow .fetch(record.key(), timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; - final KeyValue otherRecord = iter.next(); + final KeyValue otherRecord = iter.next(); context().forward( record .withValue( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java index f6f7cc8738254..cd67044ae105a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java @@ -21,15 +21,15 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -class KStreamKTableJoin implements ProcessorSupplier { +class KStreamKTableJoin implements ProcessorSupplier { - private final KeyValueMapper keyValueMapper = (key, value) -> key; - private final KTableValueGetterSupplier valueGetterSupplier; - private final ValueJoinerWithKey joiner; + private final KeyValueMapper keyValueMapper = (key, value) -> key; + private final KTableValueGetterSupplier valueGetterSupplier; + private final ValueJoinerWithKey joiner; private final boolean leftJoin; - KStreamKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, - final ValueJoinerWithKey joiner, + KStreamKTableJoin(final KTableValueGetterSupplier valueGetterSupplier, + final ValueJoinerWithKey joiner, final boolean leftJoin) { this.valueGetterSupplier = valueGetterSupplier; this.joiner = joiner; @@ -37,7 +37,7 @@ class KStreamKTableJoin implements ProcessorSupplier get() { + public Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), keyValueMapper, joiner, leftJoin); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index e1329eac1ade6..5cbc06372f0c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -30,19 +30,19 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class KStreamKTableJoinProcessor extends ContextualProcessor { +class KStreamKTableJoinProcessor extends ContextualProcessor { private static final Logger LOG = LoggerFactory.getLogger(KStreamKTableJoin.class); - private final KTableValueGetter valueGetter; - private final KeyValueMapper keyMapper; - private final ValueJoinerWithKey joiner; + private final KTableValueGetter valueGetter; + private final KeyValueMapper keyMapper; + private final ValueJoinerWithKey joiner; private final boolean leftJoin; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KStreamKTableJoinProcessor(final KTableValueGetter valueGetter, - final KeyValueMapper keyMapper, - final ValueJoinerWithKey joiner, + KStreamKTableJoinProcessor(final KTableValueGetter valueGetter, + final KeyValueMapper keyMapper, + final ValueJoinerWithKey joiner, final boolean leftJoin) { this.valueGetter = valueGetter; this.keyMapper = keyMapper; @@ -51,7 +51,7 @@ class KStreamKTableJoinProcessor extends ContextualProcess } @Override - public void init(final ProcessorContext context) { + public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -59,7 +59,7 @@ public void init(final ProcessorContext context) { } @Override - public void process(final Record record) { + public void process(final Record record) { // we do join iff the join keys are equal, thus, if {@code keyMapper} returns {@code null} we // cannot join and just ignore the record. Note for KTables, this is the same as having a null key // since keyMapper just returns the key, but for GlobalKTables we can have other keyMappers @@ -68,7 +68,7 @@ public void process(final Record record) { // an empty message (ie, there is nothing to be joined) -- this contrast SQL NULL semantics // furthermore, on left/outer joins 'null' in ValueJoiner#apply() indicates a missing record -- // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored - final K2 mappedKey = keyMapper.apply(record.key(), record.value()); + final K1 mappedKey = keyMapper.apply(record.key(), record.value()); if (mappedKey == null || record.value() == null) { LOG.warn( "Skipping record due to null join key or value. key=[{}] value=[{}] topic=[{}] partition=[{}] offset=[{}]", @@ -79,7 +79,7 @@ public void process(final Record record) { ); droppedRecordsSensor.record(); } else { - final V2 value2 = getValueOrNull(valueGetter.get(mappedKey)); + final V1 value2 = getValueOrNull(valueGetter.get(mappedKey)); if (leftJoin || value2 != null) { context().forward(record.withValue(joiner.apply(record.key(), record.value(), value2))); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java index 2e91e5b0e5dae..7ce2a1eaf8778 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableChangeProcessorSupplier.java @@ -18,9 +18,9 @@ import org.apache.kafka.streams.processor.api.ProcessorSupplier; -public interface KTableChangeProcessorSupplier extends ProcessorSupplier, KOut, Change> { +public interface KTableChangeProcessorSupplier extends ProcessorSupplier, KOut, Change> { - KTableValueGetterSupplier view(); + KTableValueGetterSupplier view(); /** * Potentially enables sending old values. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index dc7d491b4203c..8b4bd0acce13c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -263,13 +263,13 @@ public KTable filterNot(final Predicate predicate, return doFilter(predicate, renamed, materializedInternal, true); } - private KTable doMapValues(final ValueMapperWithKey mapper, + private KTable doMapValues(final ValueMapperWithKey mapper, final Named named, - final MaterializedInternal> materializedInternal) { + final MaterializedInternal> materializedInternal) { final Serde keySerde; - final Serde valueSerde; + final Serde valueSerde; final String queryableStoreName; - final StoreBuilder> storeBuilder; + final StoreBuilder> storeBuilder; if (materializedInternal != null) { // we actually do not need to generate store names at all since if it is not specified, we will not @@ -291,11 +291,11 @@ private KTable doMapValues(final ValueMapperWithKey, K, Change> processorSupplier = new KTableMapValues<>(this, mapper, queryableStoreName); + final ProcessorSupplier, K, Change> processorSupplier = new KTableMapValues<>(this, mapper, queryableStoreName); // leaving in calls to ITB until building topology with graph - final ProcessorParameters processorParameters = unsafeCastProcessorParametersToCompletelyDifferentType( + final ProcessorParameters processorParameters = unsafeCastProcessorParametersToCompletelyDifferentType( new ProcessorParameters<>(processorSupplier, name) ); final GraphNode tableNode = new TableProcessorNode<>( @@ -309,7 +309,7 @@ private KTable doMapValues(final ValueMapperWithKey( + return new KTableImpl( name, keySerde, valueSerde, @@ -322,75 +322,75 @@ private KTable doMapValues(final ValueMapperWithKey KTable mapValues(final ValueMapper mapper) { + public KTable mapValues(final ValueMapper mapper) { Objects.requireNonNull(mapper, "mapper can't be null"); return doMapValues(withKey(mapper), NamedInternal.empty(), null); } @Override - public KTable mapValues(final ValueMapper mapper, + public KTable mapValues(final ValueMapper mapper, final Named named) { Objects.requireNonNull(mapper, "mapper can't be null"); return doMapValues(withKey(mapper), named, null); } @Override - public KTable mapValues(final ValueMapperWithKey mapper) { + public KTable mapValues(final ValueMapperWithKey mapper) { Objects.requireNonNull(mapper, "mapper can't be null"); return doMapValues(mapper, NamedInternal.empty(), null); } @Override - public KTable mapValues(final ValueMapperWithKey mapper, + public KTable mapValues(final ValueMapperWithKey mapper, final Named named) { Objects.requireNonNull(mapper, "mapper can't be null"); return doMapValues(mapper, named, null); } @Override - public KTable mapValues(final ValueMapper mapper, - final Materialized> materialized) { + public KTable mapValues(final ValueMapper mapper, + final Materialized> materialized) { return mapValues(mapper, NamedInternal.empty(), materialized); } @Override - public KTable mapValues(final ValueMapper mapper, + public KTable mapValues(final ValueMapper mapper, final Named named, - final Materialized> materialized) { + final Materialized> materialized) { Objects.requireNonNull(mapper, "mapper can't be null"); Objects.requireNonNull(materialized, "materialized can't be null"); - final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized); + final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized); return doMapValues(withKey(mapper), named, materializedInternal); } @Override - public KTable mapValues(final ValueMapperWithKey mapper, - final Materialized> materialized) { + public KTable mapValues(final ValueMapperWithKey mapper, + final Materialized> materialized) { return mapValues(mapper, NamedInternal.empty(), materialized); } @Override - public KTable mapValues(final ValueMapperWithKey mapper, + public KTable mapValues(final ValueMapperWithKey mapper, final Named named, - final Materialized> materialized) { + final Materialized> materialized) { Objects.requireNonNull(mapper, "mapper can't be null"); Objects.requireNonNull(materialized, "materialized can't be null"); - final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized); + final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized); return doMapValues(mapper, named, materializedInternal); } @Override - public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, + public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, final String... stateStoreNames) { return doTransformValues(transformerSupplier, null, NamedInternal.empty(), stateStoreNames); } @Override - public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, + public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, final Named named, final String... stateStoreNames) { Objects.requireNonNull(named, "processorName can't be null"); @@ -398,33 +398,33 @@ public KTable transformValues(final ValueTransformerWithKeySupplier< } @Override - public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, - final Materialized> materialized, + public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, + final Materialized> materialized, final String... stateStoreNames) { return transformValues(transformerSupplier, materialized, NamedInternal.empty(), stateStoreNames); } @Override - public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, - final Materialized> materialized, + public KTable transformValues(final ValueTransformerWithKeySupplier transformerSupplier, + final Materialized> materialized, final Named named, final String... stateStoreNames) { Objects.requireNonNull(materialized, "materialized can't be null"); Objects.requireNonNull(named, "named can't be null"); - final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized); + final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized); return doTransformValues(transformerSupplier, materializedInternal, new NamedInternal(named), stateStoreNames); } - private KTable doTransformValues(final ValueTransformerWithKeySupplier transformerSupplier, - final MaterializedInternal> materializedInternal, + private KTable doTransformValues(final ValueTransformerWithKeySupplier transformerSupplier, + final MaterializedInternal> materializedInternal, final NamedInternal namedInternal, final String... stateStoreNames) { Objects.requireNonNull(stateStoreNames, "stateStoreNames"); final Serde keySerde; - final Serde valueSerde; + final Serde valueSerde; final String queryableStoreName; - final StoreBuilder> storeBuilder; + final StoreBuilder> storeBuilder; if (materializedInternal != null) { // don't inherit parent value serde, since this operation may change the value type, more specifically: @@ -444,12 +444,12 @@ private KTable doTransformValues(final ValueTransformerWithKeySuppli final String name = namedInternal.orElseGenerateWithPrefix(builder, TRANSFORMVALUES_NAME); - final KTableChangeProcessorSupplier processorSupplier = new KTableTransformValues<>( + final KTableChangeProcessorSupplier processorSupplier = new KTableTransformValues<>( this, transformerSupplier, queryableStoreName); - final ProcessorParameters processorParameters = unsafeCastProcessorParametersToCompletelyDifferentType( + final ProcessorParameters processorParameters = unsafeCastProcessorParametersToCompletelyDifferentType( new ProcessorParameters<>(processorSupplier, name) ); @@ -723,8 +723,8 @@ private KTable doJoin(final KTable other, final String joinThisName = renamed.suffixWithOrElseGet("-join-this", builder, JOINTHIS_NAME); final String joinOtherName = renamed.suffixWithOrElseGet("-join-other", builder, JOINOTHER_NAME); - final ProcessorParameters, ?, ?> joinThisProcessorParameters = new ProcessorParameters<>(joinThis, joinThisName); - final ProcessorParameters, ?, ?> joinOtherProcessorParameters = new ProcessorParameters<>(joinOther, joinOtherName); + final ProcessorParameters, K, Change> joinThisProcessorParameters = new ProcessorParameters<>(joinThis, joinThisName); + final ProcessorParameters, K, Change> joinOtherProcessorParameters = new ProcessorParameters<>(joinOther, joinOtherName); final Serde keySerde; final Serde valueSerde; @@ -853,12 +853,12 @@ boolean sendingOldValueEnabled() { * For now, I'm just explicitly lying about the parameterized type. */ @SuppressWarnings("unchecked") - private ProcessorParameters unsafeCastProcessorParametersToCompletelyDifferentType(final ProcessorParameters, ?, ?> kObjectProcessorParameters) { - return (ProcessorParameters) kObjectProcessorParameters; + private ProcessorParameters unsafeCastProcessorParametersToCompletelyDifferentType(final ProcessorParameters, ?, ?> kObjectProcessorParameters) { + return (ProcessorParameters) kObjectProcessorParameters; } @Override - public KTable join(final KTable other, + public KTable join(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner) { return doJoinOnForeignKey( @@ -872,7 +872,7 @@ public KTable join(final KTable other, } @Override - public KTable join(final KTable other, + public KTable join(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner, final Named named) { @@ -887,7 +887,7 @@ public KTable join(final KTable other, } @Override - public KTable join(final KTable other, + public KTable join(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner, final Materialized> materialized) { @@ -895,7 +895,7 @@ public KTable join(final KTable other, } @Override - public KTable join(final KTable other, + public KTable join(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner, final Named named, @@ -904,7 +904,7 @@ public KTable join(final KTable other, } @Override - public KTable leftJoin(final KTable other, + public KTable leftJoin(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner) { return doJoinOnForeignKey( @@ -918,7 +918,7 @@ public KTable leftJoin(final KTable other, } @Override - public KTable leftJoin(final KTable other, + public KTable leftJoin(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner, final Named named) { @@ -933,7 +933,7 @@ public KTable leftJoin(final KTable other, } @Override - public KTable leftJoin(final KTable other, + public KTable leftJoin(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner, final Named named, @@ -942,7 +942,7 @@ public KTable leftJoin(final KTable other, } @Override - public KTable leftJoin(final KTable other, + public KTable leftJoin(final KTable other, final Function foreignKeyExtractor, final ValueJoiner joiner, final Materialized> materialized) { @@ -950,7 +950,7 @@ public KTable leftJoin(final KTable other, } @SuppressWarnings("unchecked") - private KTable doJoinOnForeignKey(final KTable foreignKeyTable, + private KTable doJoinOnForeignKey(final KTable foreignKeyTable, final Function foreignKeyExtractor, final ValueJoiner joiner, final Named joinName, @@ -1143,7 +1143,7 @@ private KTable doJoinOnForeignKey(final KTable f ); // If we have a key serde, it's still valid, but we don't know the value serde, since it's the result - // of the joiner (VR). + // of the joiner (VOut). if (materializedInternal.keySerde() == null) { materializedInternal.withKeySerde(keySerde); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java index 6d730fade8313..91790b260c32f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java @@ -18,31 +18,31 @@ import org.apache.kafka.streams.kstream.ValueJoiner; -abstract class KTableKTableAbstractJoin implements KTableChangeProcessorSupplier { +abstract class KTableKTableAbstractJoin implements KTableChangeProcessorSupplier { - private final KTableImpl table1; - private final KTableImpl table2; - final KTableValueGetterSupplier valueGetterSupplier1; - final KTableValueGetterSupplier valueGetterSupplier2; - final ValueJoiner joiner; + private final KTableImpl table; + private final KTableImpl other; + final KTableValueGetterSupplier valueGetterSupplier1; + final KTableValueGetterSupplier valueGetterSupplier2; + final ValueJoiner joiner; boolean sendOldValues = false; - KTableKTableAbstractJoin(final KTableImpl table1, - final KTableImpl table2, - final ValueJoiner joiner) { - this.table1 = table1; - this.table2 = table2; - this.valueGetterSupplier1 = table1.valueGetterSupplier(); - this.valueGetterSupplier2 = table2.valueGetterSupplier(); + KTableKTableAbstractJoin(final KTableImpl table, + final KTableImpl other, + final ValueJoiner joiner) { + this.table = table; + this.other = other; + this.valueGetterSupplier1 = table.valueGetterSupplier(); + this.valueGetterSupplier2 = other.valueGetterSupplier(); this.joiner = joiner; } @Override public final boolean enableSendingOldValues(final boolean forceMaterialization) { // Table-table joins require upstream materialization: - table1.enableSendingOldValues(true); - table2.enableSendingOldValues(true); + table.enableSendingOldValues(true); + other.enableSendingOldValues(true); sendOldValues = true; return true; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java index 20b3c3a9fde69..648b21622cadc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoinValueGetterSupplier.java @@ -20,13 +20,13 @@ import java.util.HashSet; import java.util.Set; -public abstract class KTableKTableAbstractJoinValueGetterSupplier implements - KTableValueGetterSupplier { - final KTableValueGetterSupplier valueGetterSupplier1; - final KTableValueGetterSupplier valueGetterSupplier2; +public abstract class KTableKTableAbstractJoinValueGetterSupplier + implements KTableValueGetterSupplier { + final KTableValueGetterSupplier valueGetterSupplier1; + final KTableValueGetterSupplier valueGetterSupplier2; - KTableKTableAbstractJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableAbstractJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { this.valueGetterSupplier1 = valueGetterSupplier1; this.valueGetterSupplier2 = valueGetterSupplier2; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index cc91c65497666..f6bce1fc3fe5b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -32,55 +32,55 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class KTableKTableInnerJoin extends KTableKTableAbstractJoin { +class KTableKTableInnerJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableInnerJoin.class); - private final KeyValueMapper keyValueMapper = (key, value) -> key; + private final KeyValueMapper keyValueMapper = (key, value) -> key; - KTableKTableInnerJoin(final KTableImpl table1, - final KTableImpl table2, - final ValueJoiner joiner) { + KTableKTableInnerJoin(final KTableImpl table1, + final KTableImpl table2, + final ValueJoiner joiner) { super(table1, table2, joiner); } @Override - public Processor, K, Change> get() { + public Processor, K, Change> get() { return new KTableKTableJoinProcessor(valueGetterSupplier2.get()); } @Override - public KTableValueGetterSupplier view() { + public KTableValueGetterSupplier view() { return new KTableKTableInnerJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } - private class KTableKTableInnerJoinValueGetterSupplier extends - KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableInnerJoinValueGetterSupplier + extends KTableKTableAbstractJoinValueGetterSupplier { KTableKTableInnerJoinValueGetterSupplier( - final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } - public KTableValueGetter get() { + public KTableValueGetter get() { return new KTableKTableInnerJoinValueGetter(valueGetterSupplier1.get(), valueGetterSupplier2.get()); } } - private class KTableKTableJoinProcessor extends ContextualProcessor, K, Change> { + private class KTableKTableJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @Override - public void init(final ProcessorContext> context) { + public void init(final ProcessorContext> context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); @@ -88,7 +88,7 @@ public void init(final ProcessorContext> context) { } @Override - public void process(final Record> record) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record if (record.key() == null) { LOG.warn( @@ -102,12 +102,12 @@ public void process(final Record> record) { return; } - R newValue = null; + VOut newValue = null; final long resultTimestamp; - R oldValue = null; + VOut oldValue = null; - final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(record.key()); - final V2 valueRight = getValueOrNull(valueAndTimestampRight); + final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(record.key()); + final V1 valueRight = getValueOrNull(valueAndTimestampRight); if (valueRight == null) { return; } @@ -132,13 +132,13 @@ public void close() { } } - private class KTableKTableInnerJoinValueGetter implements KTableValueGetter { + private class KTableKTableInnerJoinValueGetter implements KTableValueGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableInnerJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableInnerJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @@ -151,14 +151,14 @@ public void init(final ProcessorContext con } @Override - public ValueAndTimestamp get(final K key) { - final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); - final V1 value1 = getValueOrNull(valueAndTimestamp1); + public ValueAndTimestamp get(final K key) { + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); + final V value1 = getValueOrNull(valueAndTimestamp1); if (value1 != null) { - final ValueAndTimestamp valueAndTimestamp2 = valueGetter2 + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2 .get(keyValueMapper.apply(key, value1)); - final V2 value2 = getValueOrNull(valueAndTimestamp2); + final V1 value2 = getValueOrNull(valueAndTimestamp2); if (value2 != null) { return ValueAndTimestamp.make( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 15091f05a541c..e796f2779aa18 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -32,17 +32,17 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableKTableLeftJoin extends KTableKTableAbstractJoin { +class KTableKTableLeftJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableLeftJoin.class); - KTableKTableLeftJoin(final KTableImpl table1, - final KTableImpl table2, - final ValueJoiner joiner) { - super(table1, table2, joiner); + KTableKTableLeftJoin(final KTableImpl table, + final KTableImpl other, + final ValueJoiner joiner) { + super(table, other, joiner); } @Override - public Processor, K, Change> get() { + public Processor, K, Change> get() { return new KTableKTableLeftJoinProcessor(valueGetterSupplier2.get()); } @@ -51,10 +51,10 @@ public KTableValueGetterSupplier view() { return new KTableKTableLeftJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } - private class KTableKTableLeftJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableLeftJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableLeftJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableLeftJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } @@ -64,13 +64,13 @@ public KTableValueGetter get() { } - private class KTableKTableLeftJoinProcessor extends ContextualProcessor, K, Change> { + private class KTableKTableLeftJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableLeftJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableLeftJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @@ -83,7 +83,7 @@ public void init(final ProcessorContext> context) { } @Override - public void process(final Record> record) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record if (record.key() == null) { LOG.warn( @@ -101,8 +101,8 @@ public void process(final Record> record) { final long resultTimestamp; VOut oldValue = null; - final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(record.key()); - final V2 value2 = getValueOrNull(valueAndTimestampRight); + final ValueAndTimestamp valueAndTimestampRight = valueGetter.get(record.key()); + final V1 value2 = getValueOrNull(valueAndTimestampRight); final long timestampRight; if (value2 == null) { @@ -136,11 +136,11 @@ public void close() { private class KTableKTableLeftJoinValueGetter implements KTableValueGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableLeftJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableLeftJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @@ -153,12 +153,12 @@ public void init(final ProcessorContext con @Override public ValueAndTimestamp get(final K key) { - final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); - final V1 value1 = getValueOrNull(valueAndTimestamp1); + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); + final V value1 = getValueOrNull(valueAndTimestamp1); if (value1 != null) { - final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); - final V2 value2 = getValueOrNull(valueAndTimestamp2); + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); + final V1 value2 = getValueOrNull(valueAndTimestamp2); final long resultTimestamp; if (valueAndTimestamp2 == null) { resultTimestamp = valueAndTimestamp1.timestamp(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index c35e4fd2a578b..ee78db7d69de5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -32,17 +32,17 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableKTableOuterJoin extends KTableKTableAbstractJoin { +class KTableKTableOuterJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableOuterJoin.class); - KTableKTableOuterJoin(final KTableImpl table1, - final KTableImpl table2, - final ValueJoiner joiner) { + KTableKTableOuterJoin(final KTableImpl table1, + final KTableImpl table2, + final ValueJoiner joiner) { super(table1, table2, joiner); } @Override - public Processor, K, Change> get() { + public Processor, K, Change> get() { return new KTableKTableOuterJoinProcessor(valueGetterSupplier2.get()); } @@ -51,10 +51,10 @@ public KTableValueGetterSupplier view() { return new KTableKTableOuterJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } - private class KTableKTableOuterJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableOuterJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableOuterJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableOuterJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } @@ -63,13 +63,13 @@ public KTableValueGetter get() { } } - private class KTableKTableOuterJoinProcessor extends ContextualProcessor, K, Change> { + private class KTableKTableOuterJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableOuterJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableOuterJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @@ -82,7 +82,7 @@ public void init(final ProcessorContext> context) { } @Override - public void process(final Record> record) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record if (record.key() == null) { LOG.warn( @@ -100,8 +100,8 @@ public void process(final Record> record) { final long resultTimestamp; VOut oldValue = null; - final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(record.key()); - final V2 value2 = getValueOrNull(valueAndTimestamp2); + final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(record.key()); + final V1 value2 = getValueOrNull(valueAndTimestamp2); if (value2 == null) { if (record.value().newValue == null && record.value().oldValue == null) { return; @@ -131,11 +131,11 @@ public void close() { private class KTableKTableOuterJoinValueGetter implements KTableValueGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableOuterJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableOuterJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @@ -150,8 +150,8 @@ public void init(final ProcessorContext con public ValueAndTimestamp get(final K key) { VOut newValue = null; - final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); - final V1 value1; + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); + final V value1; final long timestamp1; if (valueAndTimestamp1 == null) { value1 = null; @@ -161,8 +161,8 @@ public ValueAndTimestamp get(final K key) { timestamp1 = valueAndTimestamp1.timestamp(); } - final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); - final V2 value2; + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); + final V1 value2; final long timestamp2; if (valueAndTimestamp2 == null) { value2 = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index 37b44a9f08915..ec66fb8ea3df7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -31,17 +31,17 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KTableKTableRightJoin extends KTableKTableAbstractJoin { +class KTableKTableRightJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableRightJoin.class); - KTableKTableRightJoin(final KTableImpl table1, - final KTableImpl table2, - final ValueJoiner joiner) { - super(table1, table2, joiner); + KTableKTableRightJoin(final KTableImpl table, + final KTableImpl other, + final ValueJoiner joiner) { + super(table, other, joiner); } @Override - public Processor, K, Change> get() { + public Processor, K, Change> get() { return new KTableKTableRightJoinProcessor(valueGetterSupplier2.get()); } @@ -50,10 +50,10 @@ public KTableValueGetterSupplier view() { return new KTableKTableRightJoinValueGetterSupplier(valueGetterSupplier1, valueGetterSupplier2); } - private class KTableKTableRightJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { + private class KTableKTableRightJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier { - KTableKTableRightJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, - final KTableValueGetterSupplier valueGetterSupplier2) { + KTableKTableRightJoinValueGetterSupplier(final KTableValueGetterSupplier valueGetterSupplier1, + final KTableValueGetterSupplier valueGetterSupplier2) { super(valueGetterSupplier1, valueGetterSupplier2); } @@ -62,13 +62,13 @@ public KTableValueGetter get() { } } - private class KTableKTableRightJoinProcessor extends ContextualProcessor, K, Change> { + private class KTableKTableRightJoinProcessor extends ContextualProcessor, K, Change> { - private final KTableValueGetter valueGetter; + private final KTableValueGetter valueGetter; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; - KTableKTableRightJoinProcessor(final KTableValueGetter valueGetter) { + KTableKTableRightJoinProcessor(final KTableValueGetter valueGetter) { this.valueGetter = valueGetter; } @@ -81,7 +81,7 @@ public void init(final ProcessorContext> context) { } @Override - public void process(final Record> record) { + public void process(final Record> record) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record if (record.key() == null) { LOG.warn( @@ -99,8 +99,8 @@ public void process(final Record> record) { final long resultTimestamp; VOut oldValue = null; - final ValueAndTimestamp valueAndTimestampLeft = valueGetter.get(record.key()); - final V2 valueLeft = getValueOrNull(valueAndTimestampLeft); + final ValueAndTimestamp valueAndTimestampLeft = valueGetter.get(record.key()); + final V1 valueLeft = getValueOrNull(valueAndTimestampLeft); if (valueLeft == null) { return; } @@ -126,11 +126,11 @@ public void close() { private class KTableKTableRightJoinValueGetter implements KTableValueGetter { - private final KTableValueGetter valueGetter1; - private final KTableValueGetter valueGetter2; + private final KTableValueGetter valueGetter1; + private final KTableValueGetter valueGetter2; - KTableKTableRightJoinValueGetter(final KTableValueGetter valueGetter1, - final KTableValueGetter valueGetter2) { + KTableKTableRightJoinValueGetter(final KTableValueGetter valueGetter1, + final KTableValueGetter valueGetter2) { this.valueGetter1 = valueGetter1; this.valueGetter2 = valueGetter2; } @@ -143,12 +143,12 @@ public void init(final ProcessorContext con @Override public ValueAndTimestamp get(final K key) { - final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); - final V2 value2 = getValueOrNull(valueAndTimestamp2); + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); + final V1 value2 = getValueOrNull(valueAndTimestamp2); if (value2 != null) { - final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); - final V1 value1 = getValueOrNull(valueAndTimestamp1); + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); + final V value1 = getValueOrNull(valueAndTimestamp1); final long resultTimestamp; if (valueAndTimestamp1 == null) { resultTimestamp = valueAndTimestamp2.timestamp(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/StreamJoinedInternal.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/StreamJoinedInternal.java index 670783eeb3b6e..d2f8530232051 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/StreamJoinedInternal.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/StreamJoinedInternal.java @@ -23,10 +23,10 @@ import java.util.Map; -public class StreamJoinedInternal extends StreamJoined { +public class StreamJoinedInternal extends StreamJoined { //Needs to be public for testing - public StreamJoinedInternal(final StreamJoined streamJoined) { + public StreamJoinedInternal(final StreamJoined streamJoined) { super(streamJoined); } @@ -34,11 +34,11 @@ public Serde keySerde() { return keySerde; } - public Serde valueSerde() { + public Serde valueSerde() { return valueSerde; } - public Serde otherValueSerde() { + public Serde otherValueSerde() { return otherValueSerde; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/BaseJoinProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/BaseJoinProcessorNode.java index 9741bbe9d82e5..67de294131bea 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/BaseJoinProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/BaseJoinProcessorNode.java @@ -23,21 +23,21 @@ * Utility base class containing the common fields between * a Stream-Stream join and a Table-Table join */ -abstract class BaseJoinProcessorNode extends GraphNode { +abstract class BaseJoinProcessorNode extends GraphNode { - private final ProcessorParameters joinThisProcessorParameters; - private final ProcessorParameters joinOtherProcessorParameters; - private final ProcessorParameters joinMergeProcessorParameters; - private final ValueJoinerWithKey valueJoiner; + private final ProcessorParameters joinThisProcessorParameters; + private final ProcessorParameters joinOtherProcessorParameters; + private final ProcessorParameters joinMergeProcessorParameters; + private final ValueJoinerWithKey valueJoiner; private final String thisJoinSideNodeName; private final String otherJoinSideNodeName; BaseJoinProcessorNode(final String nodeName, - final ValueJoinerWithKey valueJoiner, - final ProcessorParameters joinThisProcessorParameters, - final ProcessorParameters joinOtherProcessorParameters, - final ProcessorParameters joinMergeProcessorParameters, + final ValueJoinerWithKey valueJoiner, + final ProcessorParameters joinThisProcessorParameters, + final ProcessorParameters joinOtherProcessorParameters, + final ProcessorParameters joinMergeProcessorParameters, final String thisJoinSideNodeName, final String otherJoinSideNodeName) { @@ -51,19 +51,19 @@ abstract class BaseJoinProcessorNode extends GraphNode { this.otherJoinSideNodeName = otherJoinSideNodeName; } - ProcessorParameters thisProcessorParameters() { + ProcessorParameters thisProcessorParameters() { return joinThisProcessorParameters; } - ProcessorParameters otherProcessorParameters() { + ProcessorParameters otherProcessorParameters() { return joinOtherProcessorParameters; } - ProcessorParameters mergeProcessorParameters() { + ProcessorParameters mergeProcessorParameters() { return joinMergeProcessorParameters; } - ValueJoinerWithKey valueJoiner() { + ValueJoinerWithKey valueJoiner() { return valueJoiner; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java index 4f43ee2a3ac2c..16f1d7bfe9c77 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals.graph; +import java.util.Arrays; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.KTableKTableJoinMerger; @@ -24,30 +25,29 @@ import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.TimestampedKeyValueStore; -import java.util.Arrays; - /** * Too much specific information to generalize so the KTable-KTable join requires a specific node. */ -public class KTableKTableJoinNode extends BaseJoinProcessorNode, Change, Change> { +public class KTableKTableJoinNode extends + BaseJoinProcessorNode, Change, Change> { private final Serde keySerde; - private final Serde valueSerde; + private final Serde valueSerde; private final String[] joinThisStoreNames; private final String[] joinOtherStoreNames; - private final StoreBuilder> storeBuilder; + private final StoreBuilder> storeBuilder; KTableKTableJoinNode(final String nodeName, - final ProcessorParameters, ?, ?> joinThisProcessorParameters, - final ProcessorParameters, ?, ?> joinOtherProcessorParameters, - final ProcessorParameters, ?, ?> joinMergeProcessorParameters, + final ProcessorParameters, ?, ?> joinThisProcessorParameters, + final ProcessorParameters, ?, ?> joinOtherProcessorParameters, + final ProcessorParameters, ?, ?> joinMergeProcessorParameters, final String thisJoinSide, final String otherJoinSide, final Serde keySerde, - final Serde valueSerde, + final Serde valueSerde, final String[] joinThisStoreNames, final String[] joinOtherStoreNames, - final StoreBuilder> storeBuilder) { + final StoreBuilder> storeBuilder) { super(nodeName, null, @@ -68,7 +68,7 @@ public Serde keySerde() { return keySerde; } - public Serde valueSerde() { + public Serde valueSerde() { return valueSerde; } @@ -88,11 +88,11 @@ public String queryableStoreName() { * The supplier which provides processor with KTable-KTable join merge functionality. */ @SuppressWarnings("unchecked") - public KTableKTableJoinMerger joinMerger() { - final KTableKTableJoinMerger> merger = + public KTableKTableJoinMerger joinMerger() { + final KTableKTableJoinMerger> merger = mergeProcessorParameters().kTableKTableJoinMergerProcessorSupplier(); // this incorrect cast should be corrected by the end of the KIP-478 implementation - return (KTableKTableJoinMerger) merger; + return (KTableKTableJoinMerger) merger; } @Override @@ -133,82 +133,92 @@ public String toString() { "} " + super.toString(); } - public static KTableKTableJoinNodeBuilder kTableKTableJoinNodeBuilder() { + public static KTableKTableJoinNodeBuilder kTableKTableJoinNodeBuilder() { return new KTableKTableJoinNodeBuilder<>(); } - public static final class KTableKTableJoinNodeBuilder { + public static final class KTableKTableJoinNodeBuilder { + private String nodeName; - private ProcessorParameters, ?, ?> joinThisProcessorParameters; - private ProcessorParameters, ?, ?> joinOtherProcessorParameters; + private ProcessorParameters, ?, ?> joinThisProcessorParameters; + private ProcessorParameters, ?, ?> joinOtherProcessorParameters; private String thisJoinSide; private String otherJoinSide; private Serde keySerde; - private Serde valueSerde; + private Serde valueSerde; private String[] joinThisStoreNames; private String[] joinOtherStoreNames; private String queryableStoreName; - private StoreBuilder> storeBuilder; + private StoreBuilder> storeBuilder; private KTableKTableJoinNodeBuilder() { } - public KTableKTableJoinNodeBuilder withNodeName(final String nodeName) { + public KTableKTableJoinNodeBuilder withNodeName(final String nodeName) { this.nodeName = nodeName; return this; } - public KTableKTableJoinNodeBuilder withJoinThisProcessorParameters(final ProcessorParameters, ?, ?> joinThisProcessorParameters) { + public KTableKTableJoinNodeBuilder withJoinThisProcessorParameters( + final ProcessorParameters, ?, ?> joinThisProcessorParameters) { this.joinThisProcessorParameters = joinThisProcessorParameters; return this; } - public KTableKTableJoinNodeBuilder withJoinOtherProcessorParameters(final ProcessorParameters, ?, ?> joinOtherProcessorParameters) { + public KTableKTableJoinNodeBuilder withJoinOtherProcessorParameters( + final ProcessorParameters, ?, ?> joinOtherProcessorParameters) { this.joinOtherProcessorParameters = joinOtherProcessorParameters; return this; } - public KTableKTableJoinNodeBuilder withThisJoinSideNodeName(final String thisJoinSide) { + public KTableKTableJoinNodeBuilder withThisJoinSideNodeName( + final String thisJoinSide) { this.thisJoinSide = thisJoinSide; return this; } - public KTableKTableJoinNodeBuilder withOtherJoinSideNodeName(final String otherJoinSide) { + public KTableKTableJoinNodeBuilder withOtherJoinSideNodeName( + final String otherJoinSide) { this.otherJoinSide = otherJoinSide; return this; } - public KTableKTableJoinNodeBuilder withKeySerde(final Serde keySerde) { + public KTableKTableJoinNodeBuilder withKeySerde(final Serde keySerde) { this.keySerde = keySerde; return this; } - public KTableKTableJoinNodeBuilder withValueSerde(final Serde valueSerde) { + public KTableKTableJoinNodeBuilder withValueSerde( + final Serde valueSerde) { this.valueSerde = valueSerde; return this; } - public KTableKTableJoinNodeBuilder withJoinThisStoreNames(final String[] joinThisStoreNames) { + public KTableKTableJoinNodeBuilder withJoinThisStoreNames( + final String[] joinThisStoreNames) { this.joinThisStoreNames = joinThisStoreNames; return this; } - public KTableKTableJoinNodeBuilder withJoinOtherStoreNames(final String[] joinOtherStoreNames) { + public KTableKTableJoinNodeBuilder withJoinOtherStoreNames( + final String[] joinOtherStoreNames) { this.joinOtherStoreNames = joinOtherStoreNames; return this; } - public KTableKTableJoinNodeBuilder withQueryableStoreName(final String queryableStoreName) { + public KTableKTableJoinNodeBuilder withQueryableStoreName( + final String queryableStoreName) { this.queryableStoreName = queryableStoreName; return this; } - public KTableKTableJoinNodeBuilder withStoreBuilder(final StoreBuilder> storeBuilder) { + public KTableKTableJoinNodeBuilder withStoreBuilder( + final StoreBuilder> storeBuilder) { this.storeBuilder = storeBuilder; return this; } - public KTableKTableJoinNode build() { + public KTableKTableJoinNode build() { return new KTableKTableJoinNode<>( nodeName, joinThisProcessorParameters, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java index c69f7433a62e4..fbfc342a2d40c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java @@ -26,25 +26,25 @@ /** * Too much information to generalize, so Stream-Stream joins are represented by a specific node. */ -public class StreamStreamJoinNode extends BaseJoinProcessorNode { +public class StreamStreamJoinNode extends BaseJoinProcessorNode { - private final ProcessorParameters thisWindowedStreamProcessorParameters; - private final ProcessorParameters otherWindowedStreamProcessorParameters; - private final StoreBuilder> thisWindowStoreBuilder; - private final StoreBuilder> otherWindowStoreBuilder; - private final Joined joined; + private final ProcessorParameters thisWindowedStreamProcessorParameters; + private final ProcessorParameters otherWindowedStreamProcessorParameters; + private final StoreBuilder> thisWindowStoreBuilder; + private final StoreBuilder> otherWindowStoreBuilder; + private final Joined joined; private StreamStreamJoinNode(final String nodeName, - final ValueJoinerWithKey valueJoiner, - final ProcessorParameters joinThisProcessorParameters, - final ProcessorParameters joinOtherProcessParameters, - final ProcessorParameters joinMergeProcessorParameters, - final ProcessorParameters thisWindowedStreamProcessorParameters, - final ProcessorParameters otherWindowedStreamProcessorParameters, - final StoreBuilder> thisWindowStoreBuilder, - final StoreBuilder> otherWindowStoreBuilder, - final Joined joined) { + final ValueJoinerWithKey valueJoiner, + final ProcessorParameters joinThisProcessorParameters, + final ProcessorParameters joinOtherProcessParameters, + final ProcessorParameters joinMergeProcessorParameters, + final ProcessorParameters thisWindowedStreamProcessorParameters, + final ProcessorParameters otherWindowedStreamProcessorParameters, + final StoreBuilder> thisWindowStoreBuilder, + final StoreBuilder> otherWindowStoreBuilder, + final Joined joined) { super(nodeName, valueJoiner, @@ -89,80 +89,80 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { topologyBuilder.addStateStore(otherWindowStoreBuilder, otherWindowedStreamProcessorName, thisProcessorName); } - public static StreamStreamJoinNodeBuilder streamStreamJoinNodeBuilder() { + public static StreamStreamJoinNodeBuilder streamStreamJoinNodeBuilder() { return new StreamStreamJoinNodeBuilder<>(); } - public static final class StreamStreamJoinNodeBuilder { + public static final class StreamStreamJoinNodeBuilder { private String nodeName; - private ValueJoinerWithKey valueJoiner; - private ProcessorParameters joinThisProcessorParameters; - private ProcessorParameters joinOtherProcessorParameters; - private ProcessorParameters joinMergeProcessorParameters; - private ProcessorParameters thisWindowedStreamProcessorParameters; - private ProcessorParameters otherWindowedStreamProcessorParameters; - private StoreBuilder> thisWindowStoreBuilder; - private StoreBuilder> otherWindowStoreBuilder; - private Joined joined; + private ValueJoinerWithKey valueJoiner; + private ProcessorParameters joinThisProcessorParameters; + private ProcessorParameters joinOtherProcessorParameters; + private ProcessorParameters joinMergeProcessorParameters; + private ProcessorParameters thisWindowedStreamProcessorParameters; + private ProcessorParameters otherWindowedStreamProcessorParameters; + private StoreBuilder> thisWindowStoreBuilder; + private StoreBuilder> otherWindowStoreBuilder; + private Joined joined; private StreamStreamJoinNodeBuilder() { } - public StreamStreamJoinNodeBuilder withValueJoiner(final ValueJoinerWithKey valueJoiner) { + public StreamStreamJoinNodeBuilder withValueJoiner(final ValueJoinerWithKey valueJoiner) { this.valueJoiner = valueJoiner; return this; } - public StreamStreamJoinNodeBuilder withJoinThisProcessorParameters(final ProcessorParameters joinThisProcessorParameters) { + public StreamStreamJoinNodeBuilder withJoinThisProcessorParameters(final ProcessorParameters joinThisProcessorParameters) { this.joinThisProcessorParameters = joinThisProcessorParameters; return this; } - public StreamStreamJoinNodeBuilder withNodeName(final String nodeName) { + public StreamStreamJoinNodeBuilder withNodeName(final String nodeName) { this.nodeName = nodeName; return this; } - public StreamStreamJoinNodeBuilder withJoinOtherProcessorParameters(final ProcessorParameters joinOtherProcessParameters) { + public StreamStreamJoinNodeBuilder withJoinOtherProcessorParameters(final ProcessorParameters joinOtherProcessParameters) { this.joinOtherProcessorParameters = joinOtherProcessParameters; return this; } - public StreamStreamJoinNodeBuilder withJoinMergeProcessorParameters(final ProcessorParameters joinMergeProcessorParameters) { + public StreamStreamJoinNodeBuilder withJoinMergeProcessorParameters(final ProcessorParameters joinMergeProcessorParameters) { this.joinMergeProcessorParameters = joinMergeProcessorParameters; return this; } - public StreamStreamJoinNodeBuilder withThisWindowedStreamProcessorParameters(final ProcessorParameters thisWindowedStreamProcessorParameters) { + public StreamStreamJoinNodeBuilder withThisWindowedStreamProcessorParameters(final ProcessorParameters thisWindowedStreamProcessorParameters) { this.thisWindowedStreamProcessorParameters = thisWindowedStreamProcessorParameters; return this; } - public StreamStreamJoinNodeBuilder withOtherWindowedStreamProcessorParameters( - final ProcessorParameters otherWindowedStreamProcessorParameters) { + public StreamStreamJoinNodeBuilder withOtherWindowedStreamProcessorParameters( + final ProcessorParameters otherWindowedStreamProcessorParameters) { this.otherWindowedStreamProcessorParameters = otherWindowedStreamProcessorParameters; return this; } - public StreamStreamJoinNodeBuilder withThisWindowStoreBuilder(final StoreBuilder> thisWindowStoreBuilder) { + public StreamStreamJoinNodeBuilder withThisWindowStoreBuilder(final StoreBuilder> thisWindowStoreBuilder) { this.thisWindowStoreBuilder = thisWindowStoreBuilder; return this; } - public StreamStreamJoinNodeBuilder withOtherWindowStoreBuilder(final StoreBuilder> otherWindowStoreBuilder) { + public StreamStreamJoinNodeBuilder withOtherWindowStoreBuilder(final StoreBuilder> otherWindowStoreBuilder) { this.otherWindowStoreBuilder = otherWindowStoreBuilder; return this; } - public StreamStreamJoinNodeBuilder withJoined(final Joined joined) { + public StreamStreamJoinNodeBuilder withJoined(final Joined joined) { this.joined = joined; return this; } - public StreamStreamJoinNode build() { + public StreamStreamJoinNode build() { return new StreamStreamJoinNode<>(nodeName, valueJoiner, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 7e2a4e87e68de..7b68d50c01c6c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -93,7 +93,8 @@ public void testKTable() { final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); table1.toStream().process(supplier); - final KTable table2 = table1.mapValues(s -> Integer.valueOf(s)); + final KTable table2 = table1 + .mapValues((ValueMapper) Integer::valueOf); table2.toStream().process(supplier); final KTable table3 = table2.filter((key, value) -> (value % 2) == 0); From 46ca917f55fd4194ddc3f49919af02802d644f93 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 15 Apr 2021 14:54:49 +0100 Subject: [PATCH 37/40] set types --- .../kstream/internals/KStreamImpl.java | 28 +++++++++---------- .../streams/kstream/internals/KTableImpl.java | 13 ++++----- 2 files changed, 20 insertions(+), 21 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index c7d6b615830ef..787b8dd763a6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -158,7 +158,7 @@ public KStream filter(final Predicate predicate, Objects.requireNonNull(named, "named can't be null"); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, FILTER_NAME); - final ProcessorParameters processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(new KStreamFilter<>(predicate, false), name); final ProcessorGraphNode filterProcessorNode = new ProcessorGraphNode<>(name, processorParameters); @@ -187,7 +187,7 @@ public KStream filterNot(final Predicate predicate, Objects.requireNonNull(named, "named can't be null"); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, FILTER_NAME); - final ProcessorParameters processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(new KStreamFilter<>(predicate, true), name); final ProcessorGraphNode filterNotProcessorNode = new ProcessorGraphNode<>(name, processorParameters); @@ -236,7 +236,7 @@ private ProcessorGraphNode internalSelectKey(final KeyValueMapper kStreamMap = new KStreamMap<>((key, value) -> new KeyValue<>(mapper.apply(key, value), value)); - final ProcessorParameters processorParameters = new ProcessorParameters<>(kStreamMap, name); + final ProcessorParameters processorParameters = new ProcessorParameters<>(kStreamMap, name); return new ProcessorGraphNode<>(name, processorParameters); } @@ -253,7 +253,7 @@ public KStream map(final KeyValueMapper processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(new KStreamMap<>(mapper), name); final ProcessorGraphNode mapProcessorNode = new ProcessorGraphNode<>(name, processorParameters); @@ -295,8 +295,8 @@ public KStream mapValues(final ValueMapperWithKey processorParameters = - new ProcessorParameters<>(new KStreamMapValues<>(valueMapperWithKey), name); + final ProcessorParameters processorParameters = new ProcessorParameters<>( + new KStreamMapValues<>(valueMapperWithKey), name); final ProcessorGraphNode mapValuesProcessorNode = new ProcessorGraphNode<>(name, processorParameters); mapValuesProcessorNode.setValueChangingOperation(true); @@ -326,7 +326,7 @@ public KStream flatMap(final KeyValueMapper processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(new KStreamFlatMap<>(mapper), name); final ProcessorGraphNode flatMapNode = new ProcessorGraphNode<>(name, processorParameters); @@ -361,7 +361,7 @@ public KStream flatMapValues(final ValueMapperWithKey processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(new KStreamFlatMapValues<>(valueMapper), name); final ProcessorGraphNode flatMapValuesNode = new ProcessorGraphNode<>(name, processorParameters); @@ -386,7 +386,7 @@ public void print(final Printed printed) { final PrintedInternal printedInternal = new PrintedInternal<>(printed); final String name = new NamedInternal(printedInternal.name()).orElseGenerateWithPrefix(builder, PRINTING_NAME); - final ProcessorParameters processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(printedInternal.build(this.name), name); final ProcessorGraphNode printNode = new ProcessorGraphNode<>(name, processorParameters); @@ -406,7 +406,7 @@ public void foreach(final ForeachAction action, Objects.requireNonNull(named, "named can't be null"); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, FOREACH_NAME); - final ProcessorParameters processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(() -> new ForeachProcessor<>(action), name); final ProcessorGraphNode foreachNode = new ProcessorGraphNode<>(name, processorParameters); @@ -426,7 +426,7 @@ public KStream peek(final ForeachAction action, Objects.requireNonNull(named, "named can't be null"); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, PEEK_NAME); - final ProcessorParameters processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(new KStreamPeek<>(action), name); final ProcessorGraphNode peekNode = new ProcessorGraphNode<>(name, processorParameters); @@ -533,7 +533,7 @@ private KStream merge(final InternalStreamsBuilder builder, allSubTopologySourceNodes.addAll(subTopologySourceNodes); allSubTopologySourceNodes.addAll(streamImpl.subTopologySourceNodes); - final ProcessorParameters processorParameters = + final ProcessorParameters processorParameters = new ProcessorParameters<>(new PassThrough<>(), name); final ProcessorGraphNode mergeNode = new ProcessorGraphNode<>(name, processorParameters); @@ -752,7 +752,7 @@ public KTable toTable(final Named named, materializedInternal.storeName(), materializedInternal.queryableStoreName() ); - final ProcessorParameters processorParameters = new ProcessorParameters<>(tableSource, name); + final ProcessorParameters> processorParameters = new ProcessorParameters<>(tableSource, name); final GraphNode tableNode = new StreamToTableNode<>( name, processorParameters, @@ -1033,7 +1033,7 @@ static > String createRepartition } final Predicate notNullKeyPredicate = (k, v) -> k != null; - final ProcessorParameters processorParameters = new ProcessorParameters<>( + final ProcessorParameters processorParameters = new ProcessorParameters<>( new KStreamFilter<>(notNullKeyPredicate, false), nullKeyFilterProcessorName ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 8b4bd0acce13c..b624e598dbc88 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -484,14 +484,13 @@ public KStream toStream(final Named named) { final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, TOSTREAM_NAME); final KStreamMapValues, V> kStreamMapValues = new KStreamMapValues<>((key, change) -> change.newValue); - final ProcessorParameters processorParameters = unsafeCastProcessorParametersToCompletelyDifferentType( - new ProcessorParameters<>(kStreamMapValues, name) - ); + final ProcessorParameters, K, V> processorParameters = new ProcessorParameters<>( + kStreamMapValues, + name); - final ProcessorGraphNode toStreamNode = new ProcessorGraphNode<>( + final ProcessorGraphNode> toStreamNode = new ProcessorGraphNode<>( name, - processorParameters - ); + processorParameters); builder.addGraphNode(this.graphNode, toStreamNode); @@ -790,7 +789,7 @@ public KGroupedTable groupBy(final KeyValueMapper, K1, V1> selectSupplier = new KTableRepartitionMap<>(this, selector); - final ProcessorParameters, ?, ?> processorParameters = new ProcessorParameters<>(selectSupplier, selectName); + final ProcessorParameters, K1, Change> processorParameters = new ProcessorParameters<>(selectSupplier, selectName); // select the aggregate key and values (old and new), it would require parent to send old values final ProcessorGraphNode> groupByMapNode = new ProcessorGraphNode<>(selectName, processorParameters); From d1685266d493c6221d1c34e25dd1035a698fd1c1 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 13 May 2021 10:38:06 +0100 Subject: [PATCH 38/40] Add support for KTable transforms with new ProcessorContext --- .../internals/KStreamFlatTransformValues.java | 4 +- .../internals/KStreamTransformValues.java | 4 +- .../internals/KTableTransformValues.java | 42 +++-- .../processor/api/ValueTransformer.java | 106 +++++++++++ ...ForwardingDisabledOldProcessorContext.java | 167 ++++++++++++++++++ .../ForwardingDisabledProcessorContext.java | 73 ++------ .../ValueTransformerWithKeyAdapter.java | 72 ++++++++ .../KStreamFlatTransformValuesTest.java | 4 +- .../KStreamKStreamOuterJoinTest.java | 50 +++--- ...ardingDisabledOldProcessorContextTest.java | 51 ++++++ ...orwardingDisabledProcessorContextTest.java | 17 +- .../RocksDBTimeOrderedWindowStoreTest.java | 2 +- 12 files changed, 475 insertions(+), 117 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/api/ValueTransformer.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/ValueTransformerWithKeyAdapter.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java index 3693b747a9cee..3f46f2d9e5e62 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java @@ -22,7 +22,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledOldProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.state.StoreBuilder; @@ -63,7 +63,7 @@ public static class KStreamFlatTransformValuesProcessor extends @Override public void init(final ProcessorContext context) { super.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledOldProcessorContext(context)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index 1d028e10105a4..522b6096a9a3a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -22,7 +22,7 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledOldProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.state.StoreBuilder; @@ -58,7 +58,7 @@ public static class KStreamTransformValuesProcessor extends @Override public void init(final ProcessorContext context) { super.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledOldProcessorContext(context)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java index ce648f33cd71c..562fa51bf50c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java @@ -16,12 +16,14 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.ValueTransformer; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.ValueTransformerWithKeyAdapter; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -46,7 +48,8 @@ class KTableTransformValues implements KTableChangeProcessorSupplier, K, Change> get() { - return new KTableTransformValuesProcessor(transformerSupplier.get()); + return new KTableTransformValuesProcessor( + ValueTransformerWithKeyAdapter.adapt(transformerSupplier.get())); } @Override @@ -61,7 +64,7 @@ public KTableValueGetterSupplier view() { public KTableValueGetter get() { return new KTableTransformValuesGetter( parentValueGetterSupplier.get(), - transformerSupplier.get()); + ValueTransformerWithKeyAdapter.adapt(transformerSupplier.get())); } @Override @@ -86,21 +89,22 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { private class KTableTransformValuesProcessor extends ContextualProcessor, K, Change> { - private final ValueTransformerWithKey valueTransformer; + + private final ValueTransformer valueTransformer; private TimestampedKeyValueStore store; private TupleChangeForwarder tupleForwarder; - private KTableTransformValuesProcessor(final ValueTransformerWithKey valueTransformer) { + private KTableTransformValuesProcessor( + final ValueTransformer valueTransformer) { this.valueTransformer = Objects.requireNonNull(valueTransformer, "valueTransformer"); } - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext> context) { super.init(context); -// valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext<>(context)); if (queryableName != null) { - store = (TimestampedKeyValueStore) context.getStateStore(queryableName); + store = context.getStateStore(queryableName); tupleForwarder = new TupleChangeForwarder<>( store, context, @@ -111,10 +115,11 @@ public void init(final ProcessorContext> context) { @Override public void process(final Record> record) { - final V1 newValue = valueTransformer.transform(record.key(), record.value().newValue); + final V1 newValue = valueTransformer.transform(record.withValue(record.value().newValue)); if (queryableName == null) { - final V1 oldValue = sendOldValues ? valueTransformer.transform(record.key(), record.value().oldValue) : null; + final V1 v1 = valueTransformer.transform(record.withValue(record.value().oldValue)); + final V1 oldValue = sendOldValues ? v1 : null; context().forward(record.withValue(new Change<>(newValue, oldValue))); } else { final V1 oldValue = sendOldValues ? getValueOrNull(store.get(record.key())) : null; @@ -131,11 +136,12 @@ public void close() { private class KTableTransformValuesGetter implements KTableValueGetter { + private final KTableValueGetter parentGetter; - private final ValueTransformerWithKey valueTransformer; + private final ValueTransformer valueTransformer; KTableTransformValuesGetter(final KTableValueGetter parentGetter, - final ValueTransformerWithKey valueTransformer) { + final ValueTransformer valueTransformer) { this.parentGetter = Objects.requireNonNull(parentGetter, "parentGetter"); this.valueTransformer = Objects.requireNonNull(valueTransformer, "valueTransformer"); } @@ -143,15 +149,19 @@ private class KTableTransformValuesGetter implements KTableValueGetter { @Override public void init(final ProcessorContext context) { parentGetter.init(context); -// valueTransformer.init(new ForwardingDisabledProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext<>(context)); } @Override public ValueAndTimestamp get(final K key) { final ValueAndTimestamp valueAndTimestamp = parentGetter.get(key); - return ValueAndTimestamp.make( - valueTransformer.transform(key, getValueOrNull(valueAndTimestamp)), - valueAndTimestamp == null ? UNKNOWN : valueAndTimestamp.timestamp()); + final long timestamp = + valueAndTimestamp == null ? UNKNOWN : valueAndTimestamp.timestamp(); + final Record record = new Record<>( + key, + getValueOrNull(valueAndTimestamp), + timestamp); + return ValueAndTimestamp.make(valueTransformer.transform(record), timestamp); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/ValueTransformer.java new file mode 100644 index 0000000000000..fae1784126c9c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/api/ValueTransformer.java @@ -0,0 +1,106 @@ +/* + * 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.processor.api; + +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Transformer; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.kstream.ValueTransformerSupplier; +import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.StateStore; + +import java.time.Duration; + +/** + * The {@code ValueTransformerWithKey} interface for stateful mapping of a value to a new value + * (with possible new type). This is a stateful record-by-record operation, i.e, {@link + * #transform(Record)} is invoked individually for each record of a stream and can access and modify + * a state that is available beyond a single call of {@link #transform(Record)} (cf. {@link + * ValueMapper} for stateless value transformation). Additionally, this {@code + * ValueTransformerWithKey} can {@link ProcessorContext#schedule(Duration, PunctuationType, + * Punctuator) schedule} a method to be {@link Punctuator#punctuate(long) called periodically} with + * the provided context. Note that the key is read-only and should not be modified, as this can lead + * to corrupt partitioning. If {@code ValueTransformerWithKey} is applied to a {@link KeyValue} pair + * record the record's key is preserved. + *

+ * Use {@link ValueTransformerWithKeySupplier} to provide new instances of {@link + * ValueTransformer} to Kafka Stream's runtime. + *

+ * If a record's key and value should be modified {@link Transformer} can be used. + * + * @param key type + * @param value type + * @param transformed value type + * @see org.apache.kafka.streams.kstream.ValueTransformer + * @see ValueTransformerWithKeySupplier + * @see KStream#transformValues(ValueTransformerSupplier, String...) + * @see KStream#transformValues(ValueTransformerWithKeySupplier, String...) + * @see Transformer + */ + +public interface ValueTransformer { + + /** + * Initialize this transformer. This is called once per instance when the topology gets + * initialized. + *

+ * The provided {@link ProcessorContext context} can be used to access topology and record meta + * data, to {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a + * method to be {@link Punctuator#punctuate(long) called periodically} and to access attached + * {@link StateStore}s. + *

+ * Note that {@link ProcessorContext} is updated in the background with the current record's + * meta data. Thus, it only contains valid record meta data when accessed within {@link + * #transform(Record)}. + *

+ * Note that using {@link ProcessorContext#forward(Record)} is not allowed within any method of + * {@code ValueTransformerWithKey} and will result in an {@link StreamsException exception}. + * + * @param context the context + * @throws IllegalStateException If store gets registered after initialization is already + * finished + * @throws StreamsException if the store's change log does not contain the partition + */ + void init(final ProcessorContext context); + + /** + * Transform the given [key and ]value to a new value. Additionally, any {@link StateStore} that + * is {@link KStream#transformValues(ValueTransformerWithKeySupplier, String...) attached} to + * this operator can be accessed and modified arbitrarily (cf. {@link + * ProcessorContext#getStateStore(String)}). + *

+ * Note, that using {@link ProcessorContext#forward(Record)} is not allowed within {@code + * transform} and will result in an {@link StreamsException exception}. + * + * @param record the value to be transformed + * @return the new value + */ + VOut transform(Record record); + + /** + * Close this processor and clean up any resources. + *

+ * It is not possible to return any new output records within {@code close()}. Using {@link + * ProcessorContext#forward(Record)} will result in an {@link StreamsException exception}. + */ + void close(); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java new file mode 100644 index 0000000000000..5627654a7f19c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java @@ -0,0 +1,167 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.To; + +import java.io.File; +import java.time.Duration; +import java.util.Map; +import java.util.Objects; + +/** + * {@code ProcessorContext} implementation that will throw on any forward call. + */ +public final class ForwardingDisabledOldProcessorContext implements ProcessorContext { + private final ProcessorContext delegate; + + private static final String EXPLANATION = "ProcessorContext#forward() is not supported from this context, " + + "as the framework must ensure the key is not changed (#forward allows changing the key on " + + "messages which are sent). Try another function, which doesn't allow the key to be changed " + + "(for example - #tranformValues)."; + + public ForwardingDisabledOldProcessorContext(final ProcessorContext delegate) { + this.delegate = Objects.requireNonNull(delegate, "delegate"); + } + + @Override + public String applicationId() { + return delegate.applicationId(); + } + + @Override + public TaskId taskId() { + return delegate.taskId(); + } + + @Override + public Serde keySerde() { + return delegate.keySerde(); + } + + @Override + public Serde valueSerde() { + return delegate.valueSerde(); + } + + @Override + public File stateDir() { + return delegate.stateDir(); + } + + @Override + public StreamsMetrics metrics() { + return delegate.metrics(); + } + + @Override + public void register(final StateStore store, + final StateRestoreCallback stateRestoreCallback) { + delegate.register(store, stateRestoreCallback); + } + + @Override + public S getStateStore(final String name) { + return delegate.getStateStore(name); + } + + @Override + @Deprecated + public Cancellable schedule(final long intervalMs, + final PunctuationType type, + final Punctuator callback) { + return delegate.schedule(intervalMs, type, callback); + } + + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { + return delegate.schedule(interval, type, callback); + } + + @Override + public void forward(final K key, final V value) { + throw new StreamsException(EXPLANATION); + } + + @Override + public void forward(final K key, final V value, final To to) { + throw new StreamsException(EXPLANATION); + } + + @Override + public void commit() { + delegate.commit(); + } + + @Override + public String topic() { + return delegate.topic(); + } + + @Override + public int partition() { + return delegate.partition(); + } + + @Override + public long offset() { + return delegate.offset(); + } + + @Override + public Headers headers() { + return delegate.headers(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + + @Override + public Map appConfigs() { + return delegate.appConfigs(); + } + + @Override + public Map appConfigsWithPrefix(final String prefix) { + return delegate.appConfigsWithPrefix(prefix); + } + + @Override + public long currentSystemTimeMs() { + return delegate.currentSystemTimeMs(); + } + + @Override + public long currentStreamTimeMs() { + return delegate.currentStreamTimeMs(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java index aedd9551631a9..fc08619b556c6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java @@ -16,36 +16,36 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.Cancellable; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; -import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.RecordMetadata; import java.io.File; import java.time.Duration; import java.util.Map; import java.util.Objects; +import java.util.Optional; /** * {@code ProcessorContext} implementation that will throw on any forward call. */ -public final class ForwardingDisabledProcessorContext implements ProcessorContext { - private final ProcessorContext delegate; +public final class ForwardingDisabledProcessorContext implements ProcessorContext { + private final ProcessorContext delegate; private static final String EXPLANATION = "ProcessorContext#forward() is not supported from this context, " + "as the framework must ensure the key is not changed (#forward allows changing the key on " + "messages which are sent). Try another function, which doesn't allow the key to be changed " + "(for example - #tranformValues)."; - public ForwardingDisabledProcessorContext(final ProcessorContext delegate) { + public ForwardingDisabledProcessorContext(final ProcessorContext delegate) { this.delegate = Objects.requireNonNull(delegate, "delegate"); } @@ -59,6 +59,11 @@ public TaskId taskId() { return delegate.taskId(); } + @Override + public Optional recordMetadata() { + return delegate.recordMetadata(); + } + @Override public Serde keySerde() { return delegate.keySerde(); @@ -79,25 +84,11 @@ public StreamsMetrics metrics() { return delegate.metrics(); } - @Override - public void register(final StateStore store, - final StateRestoreCallback stateRestoreCallback) { - delegate.register(store, stateRestoreCallback); - } - @Override public S getStateStore(final String name) { return delegate.getStateStore(name); } - @Override - @Deprecated - public Cancellable schedule(final long intervalMs, - final PunctuationType type, - final Punctuator callback) { - return delegate.schedule(intervalMs, type, callback); - } - @Override public Cancellable schedule(final Duration interval, final PunctuationType type, @@ -106,12 +97,13 @@ public Cancellable schedule(final Duration interval, } @Override - public void forward(final K key, final V value) { + public void forward(final Record record) { throw new StreamsException(EXPLANATION); } @Override - public void forward(final K key, final V value, final To to) { + public void forward(final Record record, + final String childName) { throw new StreamsException(EXPLANATION); } @@ -120,31 +112,6 @@ public void commit() { delegate.commit(); } - @Override - public String topic() { - return delegate.topic(); - } - - @Override - public int partition() { - return delegate.partition(); - } - - @Override - public long offset() { - return delegate.offset(); - } - - @Override - public Headers headers() { - return delegate.headers(); - } - - @Override - public long timestamp() { - return delegate.timestamp(); - } - @Override public Map appConfigs() { return delegate.appConfigs(); @@ -154,14 +121,4 @@ public Map appConfigs() { public Map appConfigsWithPrefix(final String prefix) { return delegate.appConfigsWithPrefix(prefix); } - - @Override - public long currentSystemTimeMs() { - return delegate.currentSystemTimeMs(); - } - - @Override - public long currentStreamTimeMs() { - return delegate.currentStreamTimeMs(); - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ValueTransformerWithKeyAdapter.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ValueTransformerWithKeyAdapter.java new file mode 100644 index 0000000000000..4ba63bd21d177 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ValueTransformerWithKeyAdapter.java @@ -0,0 +1,72 @@ +/* + * 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.processor.internals; + + +import org.apache.kafka.streams.kstream.ValueTransformerWithKey; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.api.ValueTransformer; + +public final class ValueTransformerWithKeyAdapter implements ValueTransformer { + private final ValueTransformerWithKey delegate; + private InternalProcessorContext context; + + public static ValueTransformer adapt(final ValueTransformerWithKey delegate) { + if (delegate == null) { + return null; + } else { + return new ValueTransformerWithKeyAdapter<>(delegate); + } + } + + private ValueTransformerWithKeyAdapter(final ValueTransformerWithKey delegate) { + this.delegate = delegate; + } + + @Override + public void init(final ProcessorContext context) { + // It only makes sense to use this adapter internally to Streams, in which case + // all contexts are implementations of InternalProcessorContext. + // This would fail if someone were to use this adapter in a unit test where + // the context only implements api.ProcessorContext. + this.context = (InternalProcessorContext) context; + delegate.init((org.apache.kafka.streams.processor.ProcessorContext) context); + } + + @Override + public VOut transform(final Record record) { + final ProcessorRecordContext processorRecordContext = context.recordContext(); + try { + context.setRecordContext(new ProcessorRecordContext( + record.timestamp(), + context.offset(), + context.partition(), + context.topic(), + record.headers() + )); + return delegate.transform(record.key(), record.value()); + } finally { + context.setRecordContext(processorRecordContext); + } + } + + @Override + public void close() { + delegate.close(); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java index bbc4814294a42..bf1eac2332c74 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java @@ -26,7 +26,7 @@ import org.apache.kafka.streams.kstream.internals.KStreamFlatTransformValues.KStreamFlatTransformValuesProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledOldProcessorContext; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.junit.Before; @@ -53,7 +53,7 @@ public void setUp() { @Test public void shouldInitializeFlatTransformValuesProcessor() { - valueTransformer.init(EasyMock.isA(ForwardingDisabledProcessorContext.class)); + valueTransformer.init(EasyMock.isA(ForwardingDisabledOldProcessorContext.class)); replayAll(); processor.init(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java index 9c16c59c16c1d..6fdc1c9ded7e9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -32,8 +32,8 @@ import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockOldProcessor; +import org.apache.kafka.test.MockOldProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Test; @@ -65,7 +65,7 @@ public void testOuterJoinWithInvalidSpuriousResultFixFlag() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -92,7 +92,7 @@ public void testOuterJoinDuplicatesWithFixDisabled() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -110,7 +110,7 @@ public void testOuterJoinDuplicatesWithFixDisabled() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // Only 2 window stores should be available assertEquals(2, driver.getAllStateStores().size()); @@ -137,7 +137,7 @@ public void testOuterJoinDuplicates() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -153,7 +153,7 @@ public void testOuterJoinDuplicates() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // verifies non-joined duplicates are emitted when window has closed inputTopic1.pipeInput(0, "A0", 0); @@ -195,7 +195,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -211,7 +211,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0; @@ -246,7 +246,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -262,7 +262,7 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0; @@ -297,7 +297,7 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -313,7 +313,7 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0; @@ -348,7 +348,7 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -364,7 +364,7 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0; @@ -399,7 +399,7 @@ public void testOrdering() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -415,7 +415,7 @@ public void testOrdering() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty; this should not produce any item yet // w1 = {} @@ -446,7 +446,7 @@ public void testGracePeriod() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -468,7 +468,7 @@ public void testGracePeriod() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // push one item to the primary stream; and one item in other stream; this should not produce items because there are no joins // and window has not ended @@ -536,7 +536,7 @@ public void runOuterJoin(final StreamJoined streamJoine final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -558,7 +558,7 @@ public void runOuterJoin(final StreamJoined streamJoine driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); // 2 window stores + 1 shared window store should be available assertEquals(3, driver.getAllStateStores().size()); @@ -640,7 +640,7 @@ public void testWindowing() { final KStream stream1; final KStream stream2; final KStream joined; - final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + final MockOldProcessorSupplier supplier = new MockOldProcessorSupplier<>(); stream1 = builder.stream(topic1, consumed); stream2 = builder.stream(topic2, consumed); @@ -662,7 +662,7 @@ public void testWindowing() { driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final TestInputTopic inputTopic2 = driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - final MockProcessor processor = supplier.theCapturedProcessor(); + final MockOldProcessor processor = supplier.theCapturedProcessor(); final long time = 0L; // push two items to the primary stream; the other window is empty; this should not produce items because window has not closed @@ -691,7 +691,7 @@ public void testWindowing() { private void testUpperWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, - final MockProcessor processor) { + final MockOldProcessor processor) { long time; final TestInputTopic inputTopic1 = @@ -821,7 +821,7 @@ private void testUpperWindowBound(final int[] expectedKeys, private void testLowerWindowBound(final int[] expectedKeys, final TopologyTestDriver driver, - final MockProcessor processor) { + final MockOldProcessor processor) { long time; final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer()); // push four items with smaller timestamp (before the window) to the primary stream; this should produce four left-join and no full-join items diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java new file mode 100644 index 0000000000000..44982b9ed83e5 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java @@ -0,0 +1,51 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; +import org.easymock.EasyMockRunner; +import org.easymock.Mock; +import org.easymock.MockType; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; + +import static org.junit.Assert.assertThrows; + +@RunWith(EasyMockRunner.class) +public class ForwardingDisabledOldProcessorContextTest { + @Mock(MockType.NICE) + private ProcessorContext delegate; + private ForwardingDisabledOldProcessorContext context; + + @Before + public void setUp() { + context = new ForwardingDisabledOldProcessorContext(delegate); + } + + @Test + public void shouldThrowOnForward() { + assertThrows(StreamsException.class, () -> context.forward("key", "value")); + } + + @Test + public void shouldThrowOnForwardWithTo() { + assertThrows(StreamsException.class, () -> context.forward("key", "value", To.all())); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java index 85178dedaf65d..e49276c4ad7a2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.easymock.EasyMockRunner; import org.easymock.Mock; import org.easymock.MockType; @@ -31,21 +31,16 @@ @RunWith(EasyMockRunner.class) public class ForwardingDisabledProcessorContextTest { @Mock(MockType.NICE) - private ProcessorContext delegate; - private ForwardingDisabledProcessorContext context; + private ProcessorContext delegate; + private ForwardingDisabledProcessorContext context; @Before public void setUp() { - context = new ForwardingDisabledProcessorContext(delegate); + context = new ForwardingDisabledProcessorContext<>(delegate); } @Test public void shouldThrowOnForward() { - assertThrows(StreamsException.class, () -> context.forward("key", "value")); - } - - @Test - public void shouldThrowOnForwardWithTo() { - assertThrows(StreamsException.class, () -> context.forward("key", "value", To.all())); + assertThrows(StreamsException.class, () -> context.forward(new Record<>("key", "value", 0L))); } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java index 2646c4cf472f4..95c88ed1576ce 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java @@ -59,7 +59,7 @@ public void setup() { windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); recordCollector = new MockRecordCollector(); - context = new InternalMockProcessorContext( + context = new InternalMockProcessorContext<>( baseDir, Serdes.String(), Serdes.Integer(), From 442f3517e77b3b7afd2912355c006dda98ae390b Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 13 May 2021 11:38:07 +0100 Subject: [PATCH 39/40] compose forwarding disabled processor context based on internal --- .../internals/KStreamFlatTransformValues.java | 6 +- .../internals/KStreamTransformValues.java | 8 +- .../internals/KTableTransformValues.java | 7 +- ...ForwardingDisabledOldProcessorContext.java | 167 ----------------- .../ForwardingDisabledProcessorContext.java | 170 ++++++++++++++++-- .../KStreamFlatTransformValuesTest.java | 14 +- .../internals/KTableTransformValuesTest.java | 11 +- ...ardingDisabledOldProcessorContextTest.java | 51 ------ ...orwardingDisabledProcessorContextTest.java | 3 +- 9 files changed, 190 insertions(+), 247 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java index 3f46f2d9e5e62..6c85252227d03 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValues.java @@ -22,7 +22,8 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledOldProcessorContext; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.state.StoreBuilder; @@ -60,10 +61,11 @@ public static class KStreamFlatTransformValuesProcessor extends this.valueTransformer = valueTransformer; } + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { super.init(context); - valueTransformer.init(new ForwardingDisabledOldProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext((InternalProcessorContext) context)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index 522b6096a9a3a..49b988c1a90d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -19,10 +19,11 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledOldProcessorContext; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.state.StoreBuilder; @@ -55,10 +56,11 @@ public static class KStreamTransformValuesProcessor extends this.valueTransformer = valueTransformer; } + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { super.init(context); - valueTransformer.init(new ForwardingDisabledOldProcessorContext(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext((InternalProcessorContext) context)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java index 562fa51bf50c9..3abaa46007a11 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.api.ValueTransformer; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ValueTransformerWithKeyAdapter; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -99,10 +100,11 @@ private KTableTransformValuesProcessor( this.valueTransformer = Objects.requireNonNull(valueTransformer, "valueTransformer"); } + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext> context) { super.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext<>(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext<>((InternalProcessorContext) context)); if (queryableName != null) { store = context.getStateStore(queryableName); tupleForwarder = new TupleChangeForwarder<>( @@ -146,10 +148,11 @@ private class KTableTransformValuesGetter implements KTableValueGetter { this.valueTransformer = Objects.requireNonNull(valueTransformer, "valueTransformer"); } + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { parentGetter.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext<>(context)); + valueTransformer.init(new ForwardingDisabledProcessorContext<>((InternalProcessorContext) context)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java deleted file mode 100644 index 5627654a7f19c..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContext.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * 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.processor.internals; - -import org.apache.kafka.common.header.Headers; -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.streams.StreamsMetrics; -import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.Cancellable; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.PunctuationType; -import org.apache.kafka.streams.processor.Punctuator; -import org.apache.kafka.streams.processor.StateRestoreCallback; -import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.processor.To; - -import java.io.File; -import java.time.Duration; -import java.util.Map; -import java.util.Objects; - -/** - * {@code ProcessorContext} implementation that will throw on any forward call. - */ -public final class ForwardingDisabledOldProcessorContext implements ProcessorContext { - private final ProcessorContext delegate; - - private static final String EXPLANATION = "ProcessorContext#forward() is not supported from this context, " - + "as the framework must ensure the key is not changed (#forward allows changing the key on " - + "messages which are sent). Try another function, which doesn't allow the key to be changed " - + "(for example - #tranformValues)."; - - public ForwardingDisabledOldProcessorContext(final ProcessorContext delegate) { - this.delegate = Objects.requireNonNull(delegate, "delegate"); - } - - @Override - public String applicationId() { - return delegate.applicationId(); - } - - @Override - public TaskId taskId() { - return delegate.taskId(); - } - - @Override - public Serde keySerde() { - return delegate.keySerde(); - } - - @Override - public Serde valueSerde() { - return delegate.valueSerde(); - } - - @Override - public File stateDir() { - return delegate.stateDir(); - } - - @Override - public StreamsMetrics metrics() { - return delegate.metrics(); - } - - @Override - public void register(final StateStore store, - final StateRestoreCallback stateRestoreCallback) { - delegate.register(store, stateRestoreCallback); - } - - @Override - public S getStateStore(final String name) { - return delegate.getStateStore(name); - } - - @Override - @Deprecated - public Cancellable schedule(final long intervalMs, - final PunctuationType type, - final Punctuator callback) { - return delegate.schedule(intervalMs, type, callback); - } - - @Override - public Cancellable schedule(final Duration interval, - final PunctuationType type, - final Punctuator callback) throws IllegalArgumentException { - return delegate.schedule(interval, type, callback); - } - - @Override - public void forward(final K key, final V value) { - throw new StreamsException(EXPLANATION); - } - - @Override - public void forward(final K key, final V value, final To to) { - throw new StreamsException(EXPLANATION); - } - - @Override - public void commit() { - delegate.commit(); - } - - @Override - public String topic() { - return delegate.topic(); - } - - @Override - public int partition() { - return delegate.partition(); - } - - @Override - public long offset() { - return delegate.offset(); - } - - @Override - public Headers headers() { - return delegate.headers(); - } - - @Override - public long timestamp() { - return delegate.timestamp(); - } - - @Override - public Map appConfigs() { - return delegate.appConfigs(); - } - - @Override - public Map appConfigsWithPrefix(final String prefix) { - return delegate.appConfigsWithPrefix(prefix); - } - - @Override - public long currentSystemTimeMs() { - return delegate.currentSystemTimeMs(); - } - - @Override - public long currentStreamTimeMs() { - return delegate.currentStreamTimeMs(); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java index fc08619b556c6..3707f221163e9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java @@ -16,17 +16,24 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.api.RecordMetadata; +import org.apache.kafka.streams.processor.internals.Task.TaskType; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.StoreBuilder; +import org.apache.kafka.streams.state.internals.ThreadCache; +import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener; import java.io.File; import java.time.Duration; @@ -37,15 +44,18 @@ /** * {@code ProcessorContext} implementation that will throw on any forward call. */ -public final class ForwardingDisabledProcessorContext implements ProcessorContext { - private final ProcessorContext delegate; +public final class ForwardingDisabledProcessorContext + implements InternalProcessorContext { - private static final String EXPLANATION = "ProcessorContext#forward() is not supported from this context, " - + "as the framework must ensure the key is not changed (#forward allows changing the key on " - + "messages which are sent). Try another function, which doesn't allow the key to be changed " - + "(for example - #tranformValues)."; + private final InternalProcessorContext delegate; - public ForwardingDisabledProcessorContext(final ProcessorContext delegate) { + private static final String EXPLANATION = + "ProcessorContext#forward() is not supported from this context, " + + "as the framework must ensure the key is not changed (#forward allows changing the key on " + + "messages which are sent). Try another function, which doesn't allow the key to be changed " + + "(for example - #tranformValues)."; + + public ForwardingDisabledProcessorContext(final InternalProcessorContext delegate) { this.delegate = Objects.requireNonNull(delegate, "delegate"); } @@ -80,15 +90,110 @@ public File stateDir() { } @Override - public StreamsMetrics metrics() { + public StreamsMetricsImpl metrics() { return delegate.metrics(); } + @Override + public void setSystemTimeMs(final long timeMs) { + delegate.setSystemTimeMs(timeMs); + } + + @Override + public ProcessorRecordContext recordContext() { + return delegate.recordContext(); + } + + @Override + public void setRecordContext(final ProcessorRecordContext recordContext) { + delegate.setRecordContext(recordContext); + } + + @Override + public void setCurrentNode(final ProcessorNode currentNode) { + delegate.setCurrentNode(currentNode); + } + + @Override + public ProcessorNode currentNode() { + return delegate.currentNode(); + } + + @Override + public ThreadCache cache() { + return delegate.cache(); + } + + @Override + public void initialize() { + delegate.initialize(); + } + + @Override + public void uninitialize() { + delegate.uninitialize(); + } + + @Override + public TaskType taskType() { + return delegate.taskType(); + } + + @Override + public void transitionToActive(final StreamTask streamTask, + final RecordCollector recordCollector, + final ThreadCache newCache) { + delegate.transitionToActive(streamTask, recordCollector, newCache); + } + + @Override + public void transitionToStandby(final ThreadCache newCache) { + delegate.transitionToStandby(newCache); + } + + @Override + public void registerCacheFlushListener(final String namespace, + final DirtyEntryFlushListener listener) { + delegate.registerCacheFlushListener(namespace, listener); + } + + @Override + public T getStateStore(final StoreBuilder builder) { + return delegate.getStateStore(builder); + } + + @Override + public void logChange(final String storeName, + final Bytes key, + final byte[] value, + final long timestamp) { + delegate.logChange(storeName, key, value, timestamp); + } + + @Override + public String changelogFor(final String storeName) { + return delegate.changelogFor(storeName); + } + + @Override + public void register(final StateStore store, + final StateRestoreCallback stateRestoreCallback) { + delegate.register(store, stateRestoreCallback); + } + @Override public S getStateStore(final String name) { return delegate.getStateStore(name); } + @Deprecated + @Override + public Cancellable schedule(final long intervalMs, + final PunctuationType type, + final Punctuator callback) { + return delegate.schedule(intervalMs, type, callback); + } + @Override public Cancellable schedule(final Duration interval, final PunctuationType type, @@ -96,6 +201,16 @@ public Cancellable schedule(final Duration interval, return delegate.schedule(interval, type, callback); } + @Override + public void forward(final K1 key, final V1 value) { + throw new StreamsException(EXPLANATION); + } + + @Override + public void forward(final K1 key, final V1 value, final To to) { + throw new StreamsException(EXPLANATION); + } + @Override public void forward(final Record record) { throw new StreamsException(EXPLANATION); @@ -112,6 +227,31 @@ public void commit() { delegate.commit(); } + @Override + public String topic() { + return delegate.topic(); + } + + @Override + public int partition() { + return delegate.partition(); + } + + @Override + public long offset() { + return delegate.offset(); + } + + @Override + public Headers headers() { + return delegate.headers(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + @Override public Map appConfigs() { return delegate.appConfigs(); @@ -121,4 +261,14 @@ public Map appConfigs() { public Map appConfigsWithPrefix(final String prefix) { return delegate.appConfigsWithPrefix(prefix); } + + @Override + public long currentSystemTimeMs() { + return delegate.currentSystemTimeMs(); + } + + @Override + public long currentStreamTimeMs() { + return delegate.currentSystemTimeMs(); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java index bf1eac2332c74..f4f0845814634 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java @@ -16,22 +16,22 @@ */ package org.apache.kafka.streams.kstream.internals; -import static org.junit.Assert.assertTrue; - -import java.util.Arrays; -import java.util.Collections; - import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.kstream.internals.KStreamFlatTransformValues.KStreamFlatTransformValuesProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.internals.ForwardingDisabledOldProcessorContext; +import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.junit.Before; import org.junit.Test; +import java.util.Arrays; +import java.util.Collections; + +import static org.junit.Assert.assertTrue; + public class KStreamFlatTransformValuesTest extends EasyMockSupport { private Integer inputKey; @@ -53,7 +53,7 @@ public void setUp() { @Test public void shouldInitializeFlatTransformValuesProcessor() { - valueTransformer.init(EasyMock.isA(ForwardingDisabledOldProcessorContext.class)); + valueTransformer.init(EasyMock.isA(ForwardingDisabledProcessorContext.class)); replayAll(); processor.init(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java index a55f395d99425..fb175297f626c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -82,7 +83,9 @@ public class KTableTransformValuesTest { @Mock(MockType.NICE) private KTableImpl parent; @Mock(MockType.NICE) - private InternalProcessorContext context; + private InternalProcessorContext context; + @Mock(MockType.NICE) + private ProcessorRecordContext recordContext; @Mock(MockType.NICE) private KTableValueGetterSupplier parentGetterSupplier; @Mock(MockType.NICE) @@ -210,16 +213,16 @@ public void shouldSetSendOldValuesOnParentIfNotMaterialized() { verify(parent); } - @SuppressWarnings("unchecked") @Test public void shouldTransformOnGetIfNotMaterialized() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null); + expect(context.recordContext()).andReturn(recordContext); expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); expect(parentGetterSupplier.get()).andReturn(parentGetter); - expect(parentGetter.get("Key")).andReturn(ValueAndTimestamp.make("Value", -1L)); - replay(parent, parentGetterSupplier, parentGetter); + expect(parentGetter.get("Key")).andReturn(ValueAndTimestamp.make("Value", 1L)); + replay(context, parent, parentGetterSupplier, parentGetter); final KTableValueGetter getter = transformValues.view().get(); getter.init(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java deleted file mode 100644 index 44982b9ed83e5..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledOldProcessorContextTest.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.processor.internals; - -import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; -import org.easymock.EasyMockRunner; -import org.easymock.Mock; -import org.easymock.MockType; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertThrows; - -@RunWith(EasyMockRunner.class) -public class ForwardingDisabledOldProcessorContextTest { - @Mock(MockType.NICE) - private ProcessorContext delegate; - private ForwardingDisabledOldProcessorContext context; - - @Before - public void setUp() { - context = new ForwardingDisabledOldProcessorContext(delegate); - } - - @Test - public void shouldThrowOnForward() { - assertThrows(StreamsException.class, () -> context.forward("key", "value")); - } - - @Test - public void shouldThrowOnForwardWithTo() { - assertThrows(StreamsException.class, () -> context.forward("key", "value", To.all())); - } -} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java index e49276c4ad7a2..ac2425ad986f6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java @@ -34,9 +34,10 @@ public class ForwardingDisabledProcessorContextTest { private ProcessorContext delegate; private ForwardingDisabledProcessorContext context; + @SuppressWarnings("unchecked") @Before public void setUp() { - context = new ForwardingDisabledProcessorContext<>(delegate); + context = new ForwardingDisabledProcessorContext<>((InternalProcessorContext) delegate); } @Test From 486657a2cf864de771aa9c21347ee9a169921e0c Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 13 May 2021 12:08:38 +0100 Subject: [PATCH 40/40] fix internal processor contexts on tests --- .../KStreamFlatTransformValuesTest.java | 3 ++- ...reamSessionWindowAggregateProcessorTest.java | 2 +- .../ForwardingDisabledProcessorContextTest.java | 17 +++++++++++++---- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java index f4f0845814634..b5d65d7e0a140 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.junit.Before; @@ -47,7 +48,7 @@ public void setUp() { inputKey = 1; inputValue = 10; valueTransformer = mock(ValueTransformerWithKey.class); - context = strictMock(ProcessorContext.class); + context = strictMock(InternalProcessorContext.class); processor = new KStreamFlatTransformValuesProcessor<>(valueTransformer); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index bf7d293bb911a..5598c6d7203f6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -87,7 +87,7 @@ public class KStreamSessionWindowAggregateProcessorTest { private final Processor, Change> processor = sessionAggregator.get(); private SessionStore sessionStore; private InternalMockProcessorContext, Change> context; - private Metrics metrics; + private final Metrics metrics = new Metrics(); @Before public void setup() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java index ac2425ad986f6..ab3fa2f1f8785 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContextTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.easymock.EasyMockRunner; import org.easymock.Mock; @@ -31,17 +31,26 @@ @RunWith(EasyMockRunner.class) public class ForwardingDisabledProcessorContextTest { @Mock(MockType.NICE) - private ProcessorContext delegate; + private InternalProcessorContext delegate; private ForwardingDisabledProcessorContext context; - @SuppressWarnings("unchecked") @Before public void setUp() { - context = new ForwardingDisabledProcessorContext<>((InternalProcessorContext) delegate); + context = new ForwardingDisabledProcessorContext<>(delegate); } @Test public void shouldThrowOnForward() { assertThrows(StreamsException.class, () -> context.forward(new Record<>("key", "value", 0L))); } + + @Test + public void shouldThrowOnOldForward() { + assertThrows(StreamsException.class, () -> context.forward("key", "value")); + } + + @Test + public void shouldThrowOnOldForwardWithTo() { + assertThrows(StreamsException.class, () -> context.forward("key", "value", To.all())); + } } \ No newline at end of file