You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ab...@apache.org on 2021/08/27 00:22:04 UTC

[kafka] branch trunk updated: MINOR: remove unused Properties from GraphNode#writeToTopology (#11263)

This is an automated email from the ASF dual-hosted git repository.

ableegoldman 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 d9bb988  MINOR: remove unused Properties from GraphNode#writeToTopology (#11263)
d9bb988 is described below

commit d9bb98895453fa7cdf14eb3184e904fca884f9a1
Author: A. Sophie Blee-Goldman <so...@confluent.io>
AuthorDate: Thu Aug 26 17:19:03 2021 -0700

    MINOR: remove unused Properties from GraphNode#writeToTopology (#11263)
    
    The GraphNode#writeToTopology method accepts a Properties input parameter, but never uses it in any of its implementations. We can remove this parameter to clean things up and help make it clear that writing nodes to the topology doesn't involve the app properties.
    
    Reviewers: Bruno Cadonna <ca...@confluent.io>
---
 .../kafka/streams/kstream/internals/InternalStreamsBuilder.java       | 4 ++--
 .../apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java | 4 +---
 .../org/apache/kafka/streams/kstream/internals/graph/GraphNode.java   | 3 +--
 .../kstream/internals/graph/GroupedTableOperationRepartitionNode.java | 4 +---
 .../kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java   | 3 +--
 .../streams/kstream/internals/graph/OptimizableRepartitionNode.java   | 4 +---
 .../kafka/streams/kstream/internals/graph/ProcessorGraphNode.java     | 4 +---
 .../apache/kafka/streams/kstream/internals/graph/StateStoreNode.java  | 4 +---
 .../kafka/streams/kstream/internals/graph/StatefulProcessorNode.java  | 3 +--
 .../apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java  | 4 +---
 .../kafka/streams/kstream/internals/graph/StreamSourceNode.java       | 3 +--
 .../kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java   | 3 +--
 .../kafka/streams/kstream/internals/graph/StreamTableJoinNode.java    | 3 +--
 .../kafka/streams/kstream/internals/graph/StreamToTableNode.java      | 4 +---
 .../kafka/streams/kstream/internals/graph/TableProcessorNode.java     | 3 +--
 .../apache/kafka/streams/kstream/internals/graph/TableSourceNode.java | 3 +--
 .../streams/kstream/internals/graph/UnoptimizableRepartitionNode.java | 4 +---
 .../kafka/streams/kstream/internals/graph/TableSourceNodeTest.java    | 4 +---
 18 files changed, 19 insertions(+), 45 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 03ee8bb..7f75f72 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
@@ -74,7 +74,7 @@ public class InternalStreamsBuilder implements InternalNameProvider {
 
     protected final GraphNode root = new GraphNode(TOPOLOGY_ROOT) {
         @Override
-        public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+        public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
             // no-op for root node
         }
     };
@@ -290,7 +290,7 @@ public class InternalStreamsBuilder implements InternalNameProvider {
             }
 
             if (streamGraphNode.allParentsWrittenToTopology() && !streamGraphNode.hasWrittenToTopology()) {
-                streamGraphNode.writeToTopology(internalTopologyBuilder, props);
+                streamGraphNode.writeToTopology(internalTopologyBuilder);
                 streamGraphNode.setHasWrittenToTopology(true);
             }
 
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
index 90d9efb..753e076 100644
--- 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
@@ -22,8 +22,6 @@ import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 import org.apache.kafka.streams.state.StoreBuilder;
 
-import java.util.Properties;
-
 public class GlobalStoreNode<KIn, VIn, S extends StateStore> extends StateStoreNode<S> {
 
     private final String sourceName;
@@ -49,7 +47,7 @@ public class GlobalStoreNode<KIn, VIn, S extends StateStore> extends StateStoreN
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         storeBuilder.withLoggingDisabled();
         topologyBuilder.addGlobalStore(storeBuilder,
                                        sourceName,
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java
index c55395a..76c2b5c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java
@@ -23,7 +23,6 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedHashSet;
-import java.util.Properties;
 
 public abstract class GraphNode {
 
@@ -118,7 +117,7 @@ public abstract class GraphNode {
         return this.buildPriority;
     }
 
-    public abstract void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props);
+    public abstract void writeToTopology(final InternalTopologyBuilder topologyBuilder);
 
     public boolean hasWrittenToTopology() {
         return hasWrittenToTopology;
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java
index 1117e5e..a7ba30d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java
@@ -26,8 +26,6 @@ import org.apache.kafka.streams.processor.FailOnInvalidTimestamp;
 import org.apache.kafka.streams.processor.internals.InternalTopicProperties;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 
-import java.util.Properties;
-
 public class GroupedTableOperationRepartitionNode<K, V> extends BaseRepartitionNode<K, V> {
 
 
@@ -80,7 +78,7 @@ public class GroupedTableOperationRepartitionNode<K, V> extends BaseRepartitionN
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties);
 
         topologyBuilder.addSink(
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 ac8d821..e6bd49d 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
@@ -25,7 +25,6 @@ import org.apache.kafka.streams.state.StoreBuilder;
 import org.apache.kafka.streams.state.TimestampedKeyValueStore;
 
 import java.util.Arrays;
-import java.util.Properties;
 
 /**
  * Too much specific information to generalize so the KTable-KTable join requires a specific node.
@@ -97,7 +96,7 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         final String thisProcessorName = thisProcessorParameters().processorName();
         final String otherProcessorName = otherProcessorParameters().processorName();
         final String mergeProcessorName = mergeProcessorParameters().processorName();
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java
index c7ee03d..a9693ec 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java
@@ -23,8 +23,6 @@ import org.apache.kafka.streams.processor.StreamPartitioner;
 import org.apache.kafka.streams.processor.internals.InternalTopicProperties;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 
-import java.util.Properties;
-
 public class OptimizableRepartitionNode<K, V> extends BaseRepartitionNode<K, V> {
 
     private OptimizableRepartitionNode(final String nodeName,
@@ -66,7 +64,7 @@ public class OptimizableRepartitionNode<K, V> extends BaseRepartitionNode<K, V>
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties);
 
         topologyBuilder.addProcessor(
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java
index c3dd4f6..a38f516 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java
@@ -19,8 +19,6 @@ package org.apache.kafka.streams.kstream.internals.graph;
 
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 
-import java.util.Properties;
-
 /**
  * Used to represent any type of stateless operation:
  *
@@ -57,7 +55,7 @@ public class ProcessorGraphNode<K, V> extends GraphNode {
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
 
         topologyBuilder.addProcessor(processorParameters.processorName(), processorParameters.processorSupplier(), parentNodeNames());
     }
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
index ae01580..32dc93d 100644
--- 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
@@ -20,8 +20,6 @@ import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 import org.apache.kafka.streams.state.StoreBuilder;
 
-import java.util.Properties;
-
 public class StateStoreNode<S extends StateStore> extends GraphNode {
 
     protected final StoreBuilder<S> storeBuilder;
@@ -33,7 +31,7 @@ public class StateStoreNode<S extends StateStore> extends GraphNode {
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
 
         topologyBuilder.addStateStore(storeBuilder);
     }
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 88264f9..381a88a 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
@@ -22,7 +22,6 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 import org.apache.kafka.streams.state.StoreBuilder;
 
 import java.util.Arrays;
-import java.util.Properties;
 import java.util.Set;
 import java.util.stream.Stream;
 
@@ -79,7 +78,7 @@ public class StatefulProcessorNode<K, V> extends ProcessorGraphNode<K, V> {
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
 
         final String processorName = processorParameters().processorName();
         final ProcessorSupplier<K, V, ?, ?> processorSupplier = processorParameters().processorSupplier();
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java
index 8d67ac1..f12a9e5 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java
@@ -26,8 +26,6 @@ import org.apache.kafka.streams.processor.TopicNameExtractor;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor;
 
-import java.util.Properties;
-
 public class StreamSinkNode<K, V> extends GraphNode {
 
     private final TopicNameExtractor<K, V> topicNameExtractor;
@@ -54,7 +52,7 @@ public class StreamSinkNode<K, V> extends GraphNode {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         final Serializer<K> keySerializer = producedInternal.keySerde() == null ? null : producedInternal.keySerde().serializer();
         final Serializer<V> valSerializer = producedInternal.valueSerde() == null ? null : producedInternal.valueSerde().serializer();
         final String[] parentNames = parentNodeNames();
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java
index e68a9c6..81cd569 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java
@@ -23,7 +23,6 @@ import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 
 import java.util.Collection;
-import java.util.Properties;
 import java.util.regex.Pattern;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,7 +69,7 @@ public class StreamSourceNode<K, V> extends SourceGraphNode<K, V> {
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
 
         if (topicPattern().isPresent()) {
             topologyBuilder.addSource(consumedInternal().offsetResetPolicy(),
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 7b0e5aa..40bfff0 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,7 +26,6 @@ import org.apache.kafka.streams.state.internals.KeyAndJoinSide;
 import org.apache.kafka.streams.state.internals.LeftOrRightValue;
 
 import java.util.Optional;
-import java.util.Properties;
 
 /**
  * Too much information to generalize, so Stream-Stream joins are represented by a specific node.
@@ -85,7 +84,7 @@ public class StreamStreamJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
 
     @SuppressWarnings("unchecked")
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
 
         final String thisProcessorName = thisProcessorParameters().processorName();
         final String otherProcessorName = otherProcessorParameters().processorName();
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java
index d5bd2b8..a4db1ba 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java
@@ -21,7 +21,6 @@ import org.apache.kafka.streams.processor.api.ProcessorSupplier;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 
 import java.util.Arrays;
-import java.util.Properties;
 
 /**
  * Represents a join between a KStream and a KTable or GlobalKTable
@@ -55,7 +54,7 @@ public class StreamTableJoinNode<K, V> extends GraphNode {
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         final String processorName = processorParameters.processorName();
         final ProcessorSupplier<K, V, ?, ?> processorSupplier = processorParameters.processorSupplier();
 
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 54231d3..3b0a572 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
@@ -26,8 +26,6 @@ import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.StoreBuilder;
 import org.apache.kafka.streams.state.TimestampedKeyValueStore;
 
-import java.util.Properties;
-
 /**
  * Represents a KTable convert From KStream
  */
@@ -54,7 +52,7 @@ public class StreamToTableNode<K, V> extends GraphNode {
 
     @SuppressWarnings("unchecked")
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         final StoreBuilder<TimestampedKeyValueStore<K, V>> storeBuilder =
             new TimestampedKeyValueStoreMaterializer<>((MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>>) materializedInternal).materialize();
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java
index f13631f..ff90a20 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java
@@ -23,7 +23,6 @@ import org.apache.kafka.streams.state.TimestampedKeyValueStore;
 
 import java.util.Arrays;
 import java.util.Objects;
-import java.util.Properties;
 
 public class TableProcessorNode<K, V> extends GraphNode {
 
@@ -58,7 +57,7 @@ public class TableProcessorNode<K, V> extends GraphNode {
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         final String processorName = processorParameters.processorName();
         topologyBuilder.addProcessor(processorName, processorParameters.processorSupplier(), parentNodeNames());
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
index 77ab7ad..3b35673 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
@@ -30,7 +30,6 @@ import org.apache.kafka.streams.state.StoreBuilder;
 import org.apache.kafka.streams.state.TimestampedKeyValueStore;
 
 import java.util.Collections;
-import java.util.Properties;
 
 /**
  * Used to represent either a KTable source or a GlobalKTable source. A boolean flag is used to indicate if this represents a GlobalKTable a {@link
@@ -83,7 +82,7 @@ public class TableSourceNode<K, V> extends SourceGraphNode<K, V> {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         final String topicName;
         if (topicNames().isPresent()) {
             final Iterator<String> topicNames = topicNames().get().iterator();
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java
index a231d23..daac9bd 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java
@@ -22,8 +22,6 @@ import org.apache.kafka.streams.processor.StreamPartitioner;
 import org.apache.kafka.streams.processor.internals.InternalTopicProperties;
 import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
 
-import java.util.Properties;
-
 /**
  * Repartition node that is not subject of optimization algorithm
  */
@@ -52,7 +50,7 @@ public class UnoptimizableRepartitionNode<K, V> extends BaseRepartitionNode<K, V
     }
 
     @Override
-    public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) {
+    public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
         topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties);
 
         topologyBuilder.addProcessor(
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java
index 1d74e89..66c55c0f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java
@@ -30,8 +30,6 @@ import org.powermock.api.easymock.PowerMock;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import java.util.Properties;
-
 @RunWith(PowerMockRunner.class)
 @PrepareForTest({InternalTopologyBuilder.class})
 public class TableSourceNodeTest {
@@ -73,6 +71,6 @@ public class TableSourceNodeTest {
             .build();
         tableSourceNode.reuseSourceTopicForChangeLog(shouldReuseSourceTopicForChangelog);
 
-        tableSourceNode.writeToTopology(topologyBuilder, new Properties());
+        tableSourceNode.writeToTopology(topologyBuilder);
     }
 }