From 4ca8e40e2f4ab415657d1e07ee868448802f4565 Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Fri, 15 Mar 2019 10:42:07 +0900 Subject: [PATCH] KAFKA-7502: Cleanup KTable materialization logic in a single place (#6174) This is a draft cleanup for KAFKA-7502. Here is the details: * Make KTableKTableJoinNode abstract, and define its child classes ([NonMaterialized,Materialized]KTableKTableJoinNode) instead: now, all materialization-related routines are separated into the other classes. * KTableKTableJoinNodeBuilder#build now instantiates [NonMaterialized,Materialized]KTableKTableJoinNode classes instead of KTableKTableJoinNode. Reviewers: Guozhang Wang , Bill Bejeck --- .../streams/kstream/internals/KTableImpl.java | 106 +++++------ .../internals/KTableKTableJoinMerger.java | 17 +- .../internals/graph/KTableKTableJoinNode.java | 172 +++++++++++------- 3 files changed, 170 insertions(+), 125 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 68f940c905e..d97213671e1 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 @@ -43,6 +43,7 @@ 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.state.KeyValueStore; +import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -465,28 +466,9 @@ public class KTableImpl extends AbstractStream implements KTable< final boolean rightOuter) { Objects.requireNonNull(other, "other can't be null"); Objects.requireNonNull(joiner, "joiner can't be null"); - final String internalQueryableName = materializedInternal == null ? null : materializedInternal.storeName(); - final String joinMergeName = builder.newProcessorName(MERGE_NAME); - return buildJoin( - (AbstractStream) other, - joiner, - leftOuter, - rightOuter, - joinMergeName, - internalQueryableName, - materializedInternal - ); - } - - private KTable buildJoin(final AbstractStream other, - final ValueJoiner joiner, - final boolean leftOuter, - final boolean rightOuter, - final String joinMergeName, - final String internalQueryableName, - final MaterializedInternal> materializedInternal) { - final Set allSourceNodes = ensureJoinableWith(other); + final String joinMergeName = builder.newProcessorName(MERGE_NAME); + final Set allSourceNodes = ensureJoinableWith((AbstractStream) other); if (leftOuter) { enableSendingOldValues(); @@ -495,57 +477,67 @@ public class KTableImpl extends AbstractStream implements KTable< ((KTableImpl) other).enableSendingOldValues(); } - final String joinThisName = builder.newProcessorName(JOINTHIS_NAME); - final String joinOtherName = builder.newProcessorName(JOINOTHER_NAME); - - - 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 KTableKTableJoinMerger joinMerge = new KTableKTableJoinMerger<>(joinThis, joinOther, internalQueryableName); + final String joinThisName = builder.newProcessorName(JOINTHIS_NAME); + final String joinOtherName = builder.newProcessorName(JOINOTHER_NAME); + + final ProcessorParameters> joinThisProcessorParameters = new ProcessorParameters<>(joinThis, joinThisName); + final ProcessorParameters> joinOtherProcessorParameters = new ProcessorParameters<>(joinOther, joinOtherName); - final KTableKTableJoinNode.KTableKTableJoinNodeBuilder kTableJoinNodeBuilder = KTableKTableJoinNode.kTableKTableJoinNodeBuilder(); + final Serde keySerde; + final Serde valueSerde; + final String queryableStoreName; + final StoreBuilder> storeBuilder; - // only materialize if specified in Materialized if (materializedInternal != null) { - kTableJoinNodeBuilder.withMaterializedInternal(materializedInternal); + keySerde = materializedInternal.keySerde() != null ? materializedInternal.keySerde() : this.keySerde; + valueSerde = materializedInternal.valueSerde(); + queryableStoreName = materializedInternal.storeName(); + storeBuilder = new KeyValueStoreMaterializer<>(materializedInternal).materialize(); + } else { + keySerde = this.keySerde; + valueSerde = null; + queryableStoreName = null; + storeBuilder = null; } - kTableJoinNodeBuilder.withNodeName(joinMergeName); - final ProcessorParameters> joinThisProcessorParameters = new ProcessorParameters<>(joinThis, joinThisName); - final ProcessorParameters> joinOtherProcessorParameters = new ProcessorParameters<>(joinOther, joinOtherName); - final ProcessorParameters> joinMergeProcessorParameters = new ProcessorParameters<>(joinMerge, joinMergeName); - - kTableJoinNodeBuilder.withJoinMergeProcessorParameters(joinMergeProcessorParameters) - .withJoinOtherProcessorParameters(joinOtherProcessorParameters) - .withJoinThisProcessorParameters(joinThisProcessorParameters) - .withJoinThisStoreNames(valueGetterSupplier().storeNames()) - .withJoinOtherStoreNames(((KTableImpl) other).valueGetterSupplier().storeNames()) - .withOtherJoinSideNodeName(((KTableImpl) other).name) - .withThisJoinSideNodeName(name); - - final KTableKTableJoinNode kTableKTableJoinNode = kTableJoinNodeBuilder.build(); + final KTableKTableJoinNode kTableKTableJoinNode = + KTableKTableJoinNode.kTableKTableJoinNodeBuilder() + .withNodeName(joinMergeName) + .withJoinThisProcessorParameters(joinThisProcessorParameters) + .withJoinOtherProcessorParameters(joinOtherProcessorParameters) + .withThisJoinSideNodeName(name) + .withOtherJoinSideNodeName(((KTableImpl) other).name) + .withJoinThisStoreNames(valueGetterSupplier().storeNames()) + .withJoinOtherStoreNames(((KTableImpl) other).valueGetterSupplier().storeNames()) + .withKeySerde(keySerde) + .withValueSerde(valueSerde) + .withQueryableStoreName(queryableStoreName) + .withStoreBuilder(storeBuilder) + .build(); builder.addGraphNode(this.streamsGraphNode, kTableKTableJoinNode); // we can inherit parent key serde if user do not provide specific overrides - return new KTableImpl, R>( - joinMergeName, - materializedInternal != null && materializedInternal.keySerde() != null ? materializedInternal.keySerde() : keySerde, - materializedInternal != null ? materializedInternal.valueSerde() : null, + return new KTableImpl, VR>( + kTableKTableJoinNode.nodeName(), + kTableKTableJoinNode.keySerde(), + kTableKTableJoinNode.valueSerde(), allSourceNodes, - internalQueryableName, - joinMerge, + kTableKTableJoinNode.queryableStoreName(), + kTableKTableJoinNode.joinMerger(), kTableKTableJoinNode, builder ); 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 78c1dc6f480..de380424612 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 @@ -25,7 +25,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; -class KTableKTableJoinMerger implements KTableProcessorSupplier { +public class KTableKTableJoinMerger implements KTableProcessorSupplier { private final KTableProcessorSupplier parent1; private final KTableProcessorSupplier parent2; @@ -40,6 +40,10 @@ class KTableKTableJoinMerger implements KTableProcessorSupplier { this.queryableName = queryableName; } + public String getQueryableName() { + return queryableName; + } + @Override public Processor> get() { return new KTableKTableJoinMergeProcessor(); @@ -78,6 +82,17 @@ class KTableKTableJoinMerger implements KTableProcessorSupplier { sendOldValues = true; } + public static KTableKTableJoinMerger of(final KTableProcessorSupplier parent1, + final KTableProcessorSupplier parent2) { + return of(parent1, parent2, null); + } + + public static KTableKTableJoinMerger of(final KTableProcessorSupplier parent1, + final KTableProcessorSupplier parent2, + final String queryableName) { + return new KTableKTableJoinMerger<>(parent1, parent2, queryableName); + } + private class KTableKTableJoinMergeProcessor extends AbstractProcessor> { private KeyValueStore store; private TupleForwarder tupleForwarder; 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 aeda0d9e19d..03bdda0b2d8 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,11 +17,10 @@ package org.apache.kafka.streams.kstream.internals.graph; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.kstream.internals.KeyValueStoreMaterializer; -import org.apache.kafka.streams.kstream.internals.MaterializedInternal; +import org.apache.kafka.streams.kstream.internals.KTableKTableJoinMerger; +import org.apache.kafka.streams.kstream.internals.KTableProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; @@ -33,32 +32,64 @@ import java.util.Arrays; */ public class KTableKTableJoinNode extends BaseJoinProcessorNode, Change, Change> { + private final Serde keySerde; + private final Serde valueSerde; private final String[] joinThisStoreNames; private final String[] joinOtherStoreNames; - private final MaterializedInternal> materializedInternal; + private final StoreBuilder> storeBuilder; KTableKTableJoinNode(final String nodeName, - final ValueJoiner, ? super Change, ? extends Change> valueJoiner, final ProcessorParameters> joinThisProcessorParameters, final ProcessorParameters> joinOtherProcessorParameters, final ProcessorParameters> joinMergeProcessorParameters, - final MaterializedInternal> materializedInternal, final String thisJoinSide, final String otherJoinSide, + final Serde keySerde, + final Serde valueSerde, final String[] joinThisStoreNames, - final String[] joinOtherStoreNames) { + final String[] joinOtherStoreNames, + final StoreBuilder> storeBuilder) { super(nodeName, - valueJoiner, - joinThisProcessorParameters, - joinOtherProcessorParameters, - joinMergeProcessorParameters, - thisJoinSide, - otherJoinSide); - + null, + joinThisProcessorParameters, + joinOtherProcessorParameters, + joinMergeProcessorParameters, + thisJoinSide, + otherJoinSide); + + this.keySerde = keySerde; + this.valueSerde = valueSerde; this.joinThisStoreNames = joinThisStoreNames; this.joinOtherStoreNames = joinOtherStoreNames; - this.materializedInternal = materializedInternal; + this.storeBuilder = storeBuilder; + } + + public Serde keySerde() { + return keySerde; + } + + public Serde valueSerde() { + return valueSerde; + } + + public String[] joinThisStoreNames() { + return joinThisStoreNames; + } + + public String[] joinOtherStoreNames() { + return joinOtherStoreNames; + } + + public String queryableStoreName() { + return ((KTableKTableJoinMerger) mergeProcessorParameters().processorSupplier()).getQueryableName(); + } + + /** + * The supplier which provides processor with KTable-KTable join merge functionality. + */ + public KTableKTableJoinMerger joinMerger() { + return (KTableKTableJoinMerger) mergeProcessorParameters().processorSupplier(); } @Override @@ -68,26 +99,24 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode> storeBuilder = - new KeyValueStoreMaterializer<>(materializedInternal).materialize(); + if (storeBuilder != null) { topologyBuilder.addStateStore(storeBuilder, mergeProcessorName); } } @@ -95,10 +124,9 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode KTableKTableJoinNodeBuilder kTableKTableJoinNodeBuilder() { @@ -106,23 +134,23 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode { - private String nodeName; - private String[] joinThisStoreNames; private ProcessorParameters> joinThisProcessorParameters; - private String[] joinOtherStoreNames; - private MaterializedInternal> materializedInternal; private ProcessorParameters> joinOtherProcessorParameters; - private ProcessorParameters> joinMergeProcessorParameters; - private ValueJoiner, ? super Change, ? extends Change> valueJoiner; private String thisJoinSide; private String otherJoinSide; + private Serde keySerde; + private Serde valueSerde; + private String[] joinThisStoreNames; + private String[] joinOtherStoreNames; + private String queryableStoreName; + private StoreBuilder> storeBuilder; private KTableKTableJoinNodeBuilder() { } - public KTableKTableJoinNodeBuilder withJoinThisStoreNames(final String[] joinThisStoreNames) { - this.joinThisStoreNames = joinThisStoreNames; + public KTableKTableJoinNodeBuilder withNodeName(final String nodeName) { + this.nodeName = nodeName; return this; } @@ -131,59 +159,69 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode withNodeName(final String nodeName) { - this.nodeName = nodeName; + public KTableKTableJoinNodeBuilder withJoinOtherProcessorParameters(final ProcessorParameters> joinOtherProcessorParameters) { + this.joinOtherProcessorParameters = joinOtherProcessorParameters; return this; } - public KTableKTableJoinNodeBuilder withJoinOtherStoreNames(final String[] joinOtherStoreNames) { - this.joinOtherStoreNames = joinOtherStoreNames; + public KTableKTableJoinNodeBuilder withThisJoinSideNodeName(final String thisJoinSide) { + this.thisJoinSide = thisJoinSide; return this; } - public KTableKTableJoinNodeBuilder withJoinOtherProcessorParameters(final ProcessorParameters> joinOtherProcessorParameters) { - this.joinOtherProcessorParameters = joinOtherProcessorParameters; + public KTableKTableJoinNodeBuilder withOtherJoinSideNodeName(final String otherJoinSide) { + this.otherJoinSide = otherJoinSide; return this; } - public KTableKTableJoinNodeBuilder withJoinMergeProcessorParameters(final ProcessorParameters> joinMergeProcessorParameters) { - this.joinMergeProcessorParameters = joinMergeProcessorParameters; + public KTableKTableJoinNodeBuilder withKeySerde(final Serde keySerde) { + this.keySerde = keySerde; return this; } - public KTableKTableJoinNodeBuilder withValueJoiner(final ValueJoiner, ? super Change, ? extends Change> valueJoiner) { - this.valueJoiner = valueJoiner; + public KTableKTableJoinNodeBuilder withValueSerde(final Serde valueSerde) { + this.valueSerde = valueSerde; return this; } - public KTableKTableJoinNodeBuilder withThisJoinSideNodeName(final String thisJoinSide) { - this.thisJoinSide = thisJoinSide; + public KTableKTableJoinNodeBuilder withJoinThisStoreNames(final String[] joinThisStoreNames) { + this.joinThisStoreNames = joinThisStoreNames; return this; } - public KTableKTableJoinNodeBuilder withOtherJoinSideNodeName(final String otherJoinSide) { - this.otherJoinSide = otherJoinSide; + public KTableKTableJoinNodeBuilder withJoinOtherStoreNames(final String[] joinOtherStoreNames) { + this.joinOtherStoreNames = joinOtherStoreNames; return this; } - public KTableKTableJoinNodeBuilder withMaterializedInternal( - final MaterializedInternal> materializedInternal) { - this.materializedInternal = materializedInternal; + public KTableKTableJoinNodeBuilder withQueryableStoreName(final String queryableStoreName) { + this.queryableStoreName = queryableStoreName; return this; } - public KTableKTableJoinNode build() { + public KTableKTableJoinNodeBuilder withStoreBuilder(final StoreBuilder> storeBuilder) { + this.storeBuilder = storeBuilder; + return this; + } + @SuppressWarnings("unchecked") + public KTableKTableJoinNode build() { return new KTableKTableJoinNode<>(nodeName, - valueJoiner, - joinThisProcessorParameters, - joinOtherProcessorParameters, - joinMergeProcessorParameters, - materializedInternal, - thisJoinSide, - otherJoinSide, - joinThisStoreNames, - joinOtherStoreNames); + joinThisProcessorParameters, + joinOtherProcessorParameters, + new ProcessorParameters<>( + KTableKTableJoinMerger.of( + (KTableProcessorSupplier) (joinThisProcessorParameters.processorSupplier()), + (KTableProcessorSupplier) (joinOtherProcessorParameters.processorSupplier()), + queryableStoreName), + nodeName), + thisJoinSide, + otherJoinSide, + keySerde, + valueSerde, + joinThisStoreNames, + joinOtherStoreNames, + storeBuilder); } } }