kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From guozh...@apache.org
Subject [kafka] branch trunk updated: MINOR: Follow up for KAFKA-6761 graph should add stores for consistency (#5453)
Date Wed, 08 Aug 2018 22:41:10 GMT
This is an automated email from the ASF dual-hosted git repository.

guozhang pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 59ae734  MINOR: Follow up for KAFKA-6761 graph should add stores for consistency
(#5453)
59ae734 is described below

commit 59ae73482d17f1bd19a27afd4c8624747b10274e
Author: Bill Bejeck <bbejeck@gmail.com>
AuthorDate: Wed Aug 8 18:41:02 2018 -0400

    MINOR: Follow up for KAFKA-6761 graph should add stores for consistency (#5453)
    
    While working on 4th PR, I noticed that I had missed adding stores via the graph vs. directly
via the InternalStreamsBuilder. Probably ok to do so, but we should be consistent.
    
    Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
---
 .../kstream/internals/InternalStreamsBuilder.java  | 23 +++----
 .../kstream/internals/graph/GlobalStoreNode.java   | 76 ++++++++++++++++++++++
 .../kstream/internals/graph/StateStoreNode.java    | 47 +++++++++++++
 3 files changed, 135 insertions(+), 11 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 2e55e9f..8895591 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
@@ -21,7 +21,9 @@ import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.kstream.GlobalKTable;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.internals.graph.GlobalStoreNode;
 import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+import org.apache.kafka.streams.kstream.internals.graph.StateStoreNode;
 import org.apache.kafka.streams.kstream.internals.graph.StreamSourceNode;
 import org.apache.kafka.streams.kstream.internals.graph.StreamsGraphNode;
 import org.apache.kafka.streams.kstream.internals.graph.TableSourceNode;
@@ -167,7 +169,7 @@ public class InternalStreamsBuilder implements InternalNameProvider {
     }
 
     public synchronized void addStateStore(final StoreBuilder builder) {
-        internalTopologyBuilder.addStateStore(builder);
+        addGraphNode(root, new StateStoreNode(builder));
     }
 
     public synchronized void addGlobalStore(final StoreBuilder<KeyValueStore> storeBuilder,
@@ -176,16 +178,15 @@ public class InternalStreamsBuilder implements InternalNameProvider
{
                                             final ConsumedInternal consumed,
                                             final String processorName,
                                             final ProcessorSupplier stateUpdateSupplier)
{
-        // explicitly disable logging for global stores
-        storeBuilder.withLoggingDisabled();
-        internalTopologyBuilder.addGlobalStore(storeBuilder,
-                                               sourceName,
-                                               consumed.timestampExtractor(),
-                                               consumed.keyDeserializer(),
-                                               consumed.valueDeserializer(),
-                                               topic,
-                                               processorName,
-                                               stateUpdateSupplier);
+
+        final StreamsGraphNode globalStoreNode = new GlobalStoreNode(storeBuilder,
+                                                                     sourceName,
+                                                                     topic,
+                                                                     consumed,
+                                                                     processorName,
+                                                                     stateUpdateSupplier);
+
+        addGraphNode(root, globalStoreNode);
     }
 
     public synchronized void addGlobalStore(final StoreBuilder<KeyValueStore> storeBuilder,
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java
new file mode 100644
index 0000000..a844de6
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream.internals.graph;
+
+import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.StoreBuilder;
+
+public class GlobalStoreNode extends StateStoreNode {
+
+
+    private final String sourceName;
+    private final String topic;
+    private final ConsumedInternal consumed;
+    private final String processorName;
+    private final ProcessorSupplier stateUpdateSupplier;
+
+
+    public GlobalStoreNode(final StoreBuilder<KeyValueStore> storeBuilder,
+                           final String sourceName,
+                           final String topic,
+                           final ConsumedInternal consumed,
+                           final String processorName,
+                           final ProcessorSupplier stateUpdateSupplier) {
+
+        super(storeBuilder);
+        this.sourceName = sourceName;
+        this.topic = topic;
+        this.consumed = consumed;
+        this.processorName = processorName;
+        this.stateUpdateSupplier = stateUpdateSupplier;
+    }
+
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
+        storeBuilder.withLoggingDisabled();
+        topologyBuilder.addGlobalStore(storeBuilder,
+                                       sourceName,
+                                       consumed.timestampExtractor(),
+                                       consumed.keyDeserializer(),
+                                       consumed.valueDeserializer(),
+                                       topic,
+                                       processorName,
+                                       stateUpdateSupplier);
+
+    }
+
+
+    @Override
+    public String toString() {
+        return "GlobalStoreNode{" +
+               "sourceName='" + sourceName + '\'' +
+               ", topic='" + topic + '\'' +
+               ", processorName='" + processorName + '\'' +
+               "} ";
+    }
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StateStoreNode.java
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StateStoreNode.java
new file mode 100644
index 0000000..a034106
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StateStoreNode.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream.internals.graph;
+
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
+import org.apache.kafka.streams.state.StoreBuilder;
+
+public class StateStoreNode extends StreamsGraphNode {
+
+    protected final StoreBuilder storeBuilder;
+
+    public StateStoreNode(final StoreBuilder storeBuilder) {
+        super(storeBuilder.toString(), false);
+
+        this.storeBuilder = storeBuilder;
+    }
+
+    @Override
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
+
+        topologyBuilder.addStateStore(storeBuilder);
+    }
+
+    @Override
+    public String toString() {
+        return "StateStoreNode{" +
+               " name='" + storeBuilder.name() +  '\'' +
+               ", logConfig=" + storeBuilder.logConfig() +
+               ", loggingEnabled='" + storeBuilder.loggingEnabled() + '\'' +
+               "} ";
+    }
+}


Mime
View raw message