You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/08/10 03:39:49 UTC

[GitHub] [kafka] vvcephei commented on a change in pull request #9148: KAFKA-10379: Implement the KIP-478 StreamBuilder#addGlobalStore()

vvcephei commented on a change in pull request #9148:
URL: https://github.com/apache/kafka/pull/9148#discussion_r467671774



##########
File path: streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
##########
@@ -499,7 +500,7 @@ public synchronized StreamsBuilder addStateStore(final StoreBuilder<?> builder)
             topic,
             new ConsumedInternal<>(consumed),
             processorName,
-            stateUpdateSupplier
+            () -> ProcessorAdapter.adapt(stateUpdateSupplier.get())

Review comment:
       Adapt the old API to the new one.

##########
File path: streams/src/main/java/org/apache/kafka/streams/Topology.java
##########
@@ -721,7 +722,7 @@ public synchronized Topology addStateStore(final StoreBuilder<?> storeBuilder,
             valueDeserializer,
             topic,
             processorName,
-            stateUpdateSupplier
+            () -> ProcessorAdapter.adapt(stateUpdateSupplier.get())

Review comment:
       Also here, adapting the old API to the new one.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
##########
@@ -201,12 +200,12 @@ public synchronized void addStateStore(final StoreBuilder<?> builder) {
         addGraphNode(root, new StateStoreNode<>(builder));
     }
 
-    public synchronized <K, V> void addGlobalStore(final StoreBuilder<?> storeBuilder,
-                                                   final String sourceName,
-                                                   final String topic,
-                                                   final ConsumedInternal<K, V> consumed,
-                                                   final String processorName,
-                                                   final ProcessorSupplier<K, V> stateUpdateSupplier) {
+    public synchronized <KIn, VIn> void addGlobalStore(final StoreBuilder<?> storeBuilder,
+                                                       final String sourceName,
+                                                       final String topic,
+                                                       final ConsumedInternal<KIn, VIn> consumed,
+                                                       final String processorName,
+                                                       final org.apache.kafka.streams.processor.api.ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier) {

Review comment:
       For the internal builder, just directly change to the new API.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
##########
@@ -89,14 +90,16 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
             new TimestampedKeyValueStoreMaterializer<>((MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>>) materializedInternal).materialize();
 
         if (isGlobalKTable) {
-            topologyBuilder.addGlobalStore(storeBuilder,
-                                           sourceName,
-                                           consumedInternal().timestampExtractor(),
-                                           consumedInternal().keyDeserializer(),
-                                           consumedInternal().valueDeserializer(),
-                                           topicName,
-                                           processorParameters.processorName(),
-                                           processorParameters.processorSupplier());
+            topologyBuilder.addGlobalStore(
+                storeBuilder,
+                sourceName,
+                consumedInternal().timestampExtractor(),
+                consumedInternal().keyDeserializer(),
+                consumedInternal().valueDeserializer(),
+                topicName,
+                processorParameters.processorName(),
+                () -> ProcessorAdapter.adapt(processorParameters.processorSupplier().get())

Review comment:
       For now, just adapt. Later this whole node will get converted to the new API.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -192,16 +192,23 @@ private boolean isWindowStore() {
     }
 
     private static class ProcessorNodeFactory<KIn, VIn, KOut, VOut> extends NodeFactory<KIn, VIn, KOut, VOut> {
-        private final ProcessorSupplier<KIn, VIn> supplier;
+        private final org.apache.kafka.streams.processor.api.ProcessorSupplier<KIn, VIn, KOut, VOut> supplier;
         private final Set<String> stateStoreNames = new HashSet<>();
 
         ProcessorNodeFactory(final String name,
                              final String[] predecessors,
-                             final ProcessorSupplier<KIn, VIn> supplier) {
+                             final org.apache.kafka.streams.processor.api.ProcessorSupplier<KIn, VIn, KOut, VOut> supplier) {

Review comment:
       We need to create a factory for the global node, so I'm adding the constructor for the new API, and also converting the internals here.

##########
File path: 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",
-            new KTableSource<>(GLOBAL_STORE_NAME, GLOBAL_STORE_NAME));
+            () -> ProcessorAdapter.adapt(new KTableSource<>(GLOBAL_STORE_NAME, GLOBAL_STORE_NAME).get()));

Review comment:
       Adapting for now. Once the Source is converted, this line will revert to the original.

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
##########
@@ -270,14 +271,15 @@ public void testPatternSourceTopicsWithGlobalTopics() {
         builder.addSource(null, "source-1", null, null, null, Pattern.compile("topic-1"));
         builder.addSource(null, "source-2", null, null, null, Pattern.compile("topic-2"));
         builder.addGlobalStore(
-                new MockKeyValueStoreBuilder("global-store", false).withLoggingDisabled(),
-                "globalSource",
-                null,
-                null,
-                null,
-                "globalTopic",
-                "global-processor",
-                new MockProcessorSupplier<>());
+            new MockKeyValueStoreBuilder("global-store", false).withLoggingDisabled(),
+            "globalSource",
+            null,
+            null,
+            null,
+            "globalTopic",
+            "global-processor",
+            new MockApiProcessorSupplier<>()

Review comment:
       Using the new API. some extra lines changed when I cleaned up the indentation formatting.

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
##########
@@ -36,9 +36,8 @@
 import org.apache.kafka.streams.state.Stores;
 import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
 import org.apache.kafka.test.IntegrationTest;
-import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockApiProcessorSupplier;

Review comment:
       I went ahead and just converted the mock processor to the new API. I gave it a temporary name. Eventually, the old one will be unused, and we can delete the old one and rename the new one back to `MockProcessorSupplier`.

##########
File path: streams/src/test/java/org/apache/kafka/test/MockApiProcessor.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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 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.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+
+public class MockApiProcessor<KIn, VIn, KOut, VOut> implements Processor<KIn, VIn, KOut, VOut> {

Review comment:
       As noted earlier, I'm copying this class from `MockProcesor` to convert it to the new API (to avoid disturbing too much code at once).

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java
##########
@@ -150,13 +149,21 @@ public void setup() {
         ((InternalProcessorContext) context).transitionToActive(task, null, null);
         EasyMock.expect(task.recordCollector()).andStubReturn(recordCollector);
 
-        context.setCurrentNode(new ProcessorNode<String, Long, Object, Object>("fake", null,
-            new HashSet<>(asList(
-                "LocalKeyValueStore",
-                "LocalTimestampedKeyValueStore",
-                "LocalWindowStore",
-                "LocalTimestampedWindowStore",
-                "LocalSessionStore"))));
+        context.setCurrentNode(
+            new ProcessorNode<>(
+                "fake",
+                (org.apache.kafka.streams.processor.api.Processor<String, Long, Object, Object>) null,

Review comment:
       This is all that changed. I had to cast the null to resolve the right overload.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
##########
@@ -54,7 +54,19 @@
     private Sensor createSensor;
 
     public ProcessorNode(final String name) {
-        this(name, null, null);
+        this(name, (Processor<KIn, VIn, KOut, VOut>) null, null);
+    }
+
+    public ProcessorNode(final String name,
+                         final Processor<KIn, VIn, KOut, VOut> processor,
+                         final Set<String> stateStores) {
+
+        this.name = name;
+        this.processor = processor;
+        this.children = new ArrayList<>();
+        this.childByName = new HashMap<>();
+        this.stateStores = stateStores;
+        this.time = new SystemTime();

Review comment:
       Adding the constructor for the new API. The old constructor still exists and still functions to adapt the old API. Eventually, the old constructor will become unused.

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
##########
@@ -176,12 +176,33 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) {
    *
    * @see `org.apache.kafka.streams.StreamsBuilder#addGlobalStore`
    */
+  @deprecated(
+    "Use #addGlobalStore(StoreBuilder, String, Consumed, org.apache.kafka.streams.processor.api.ProcessorSupplier) instead.",
+    "2.7.0"
+  )
   def addGlobalStore[K, V](storeBuilder: StoreBuilder[_ <: StateStore],
                            topic: String,
                            consumed: Consumed[K, V],
                            stateUpdateSupplier: ProcessorSupplier[K, V]): StreamsBuilderJ =
     inner.addGlobalStore(storeBuilder, topic, consumed, stateUpdateSupplier)
 
+  /**
+   * Adds a global `StateStore` to the topology. Global stores should not be added to `Processor`, `Transformer`,
+   * or `ValueTransformer` (in contrast to regular stores).
+   * <p>
+   * It is not required to connect a global store to `Processor`, `Transformer`, or `ValueTransformer`;
+   * those have read-only access to all global stores by default.
+   *
+   * @see `org.apache.kafka.streams.StreamsBuilder#addGlobalStore`
+   */
+  def addGlobalStore[K, V](

Review comment:
       Keeping the Scala API in sync with the Java one.

##########
File path: streams/src/test/java/org/apache/kafka/test/MockApiProcessorSupplier.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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 org.apache.kafka.streams.processor.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorSupplier;
+import org.apache.kafka.streams.processor.PunctuationType;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class MockApiProcessorSupplier<KIn, VIn, KOut, VOut> implements ProcessorSupplier<KIn, VIn, KOut, VOut> {

Review comment:
       As noted earlier, I'm copying this class from `MockProcesorSupplier` to convert it to the new API (to avoid disturbing too much code at once).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org