@ -17,11 +17,10 @@
@@ -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.KeyValueStoreMaterializ er ;
import org.apache.kafka.streams.kstream.internals.MaterializedInternal ;
import org.apache.kafka.streams.kstream.internals.KTableKTableJoinMerg er ;
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;
@@ -33,32 +32,64 @@ import java.util.Arrays;
* /
public class KTableKTableJoinNode < K , V1 , V2 , VR > extends BaseJoinProcessorNode < K , Change < V1 > , Change < V2 > , Change < VR > > {
private final Serde < K > keySerde ;
private final Serde < VR > valueSerde ;
private final String [ ] joinThisStoreNames ;
private final String [ ] joinOtherStoreNames ;
private final MaterializedInternal < K , VR , K eyValueStore< Bytes , byte [ ] > > materializedInternal ;
private final StoreBuilder < KeyValueStore < K , VR > > storeBuilder ;
KTableKTableJoinNode ( final String nodeName ,
final ValueJoiner < ? super Change < V1 > , ? super Change < V2 > , ? extends Change < VR > > valueJoiner ,
final ProcessorParameters < K , Change < V1 > > joinThisProcessorParameters ,
final ProcessorParameters < K , Change < V2 > > joinOtherProcessorParameters ,
final ProcessorParameters < K , Change < VR > > joinMergeProcessorParameters ,
final MaterializedInternal < K , VR , KeyValueStore < Bytes , byte [ ] > > materializedInternal ,
final String thisJoinSide ,
final String otherJoinSide ,
final Serde < K > keySerde ,
final Serde < VR > valueSerde ,
final String [ ] joinThisStoreNames ,
final String [ ] joinOtherStoreNames ) {
final String [ ] joinOtherStoreNames ,
final StoreBuilder < KeyValueStore < K , VR > > 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 < K > keySerde ( ) {
return keySerde ;
}
public Serde < VR > 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 < K , VR > joinMerger ( ) {
return ( KTableKTableJoinMerger < K , VR > ) mergeProcessorParameters ( ) . processorSupplier ( ) ;
}
@Override
@ -68,26 +99,24 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
@@ -68,26 +99,24 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
final String mergeProcessorName = mergeProcessorParameters ( ) . processorName ( ) ;
topologyBuilder . addProcessor ( thisProcessorName ,
thisProcessorParameters ( ) . processorSupplier ( ) ,
thisJoinSideNodeName ( ) ) ;
thisProcessorParameters ( ) . processorSupplier ( ) ,
thisJoinSideNodeName ( ) ) ;
topologyBuilder . addProcessor ( otherProcessorName ,
otherProcessorParameters ( ) . processorSupplier ( ) ,
otherJoinSideNodeName ( ) ) ;
otherProcessorParameters ( ) . processorSupplier ( ) ,
otherJoinSideNodeName ( ) ) ;
topologyBuilder . addProcessor ( mergeProcessorName ,
mergeProcessorParameters ( ) . processorSupplier ( ) ,
thisProcessorName ,
otherProcessorName ) ;
mergeProcessorParameters ( ) . processorSupplier ( ) ,
thisProcessorName ,
otherProcessorName ) ;
topologyBuilder . connectProcessorAndStateStores ( thisProcessorName ,
joinOtherStoreNames ) ;
joinOtherStoreNames ) ;
topologyBuilder . connectProcessorAndStateStores ( otherProcessorName ,
joinThisStoreNames ) ;
joinThisStoreNames ) ;
if ( materializedInternal ! = null ) {
final StoreBuilder < KeyValueStore < K , VR > > storeBuilder =
new KeyValueStoreMaterializer < > ( materializedInternal ) . materialize ( ) ;
if ( storeBuilder ! = null ) {
topologyBuilder . addStateStore ( storeBuilder , mergeProcessorName ) ;
}
}
@ -95,10 +124,9 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
@@ -95,10 +124,9 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
@Override
public String toString ( ) {
return "KTableKTableJoinNode{" +
"joinThisStoreNames=" + Arrays . toString ( joinThisStoreNames ) +
", joinOtherStoreNames=" + Arrays . toString ( joinOtherStoreNames ) +
", materializedInternal=" + materializedInternal +
"} " + super . toString ( ) ;
"joinThisStoreNames=" + Arrays . toString ( joinThisStoreNames ( ) ) +
", joinOtherStoreNames=" + Arrays . toString ( joinOtherStoreNames ( ) ) +
"} " + super . toString ( ) ;
}
public static < K , V1 , V2 , VR > KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > kTableKTableJoinNodeBuilder ( ) {
@ -106,23 +134,23 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
@@ -106,23 +134,23 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
}
public static final class KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > {
private String nodeName ;
private String [ ] joinThisStoreNames ;
private ProcessorParameters < K , Change < V1 > > joinThisProcessorParameters ;
private String [ ] joinOtherStoreNames ;
private MaterializedInternal < K , VR , KeyValueStore < Bytes , byte [ ] > > materializedInternal ;
private ProcessorParameters < K , Change < V2 > > joinOtherProcessorParameters ;
private ProcessorParameters < K , Change < VR > > joinMergeProcessorParameters ;
private ValueJoiner < ? super Change < V1 > , ? super Change < V2 > , ? extends Change < VR > > valueJoiner ;
private String thisJoinSide ;
private String otherJoinSide ;
private Serde < K > keySerde ;
private Serde < VR > valueSerde ;
private String [ ] joinThisStoreNames ;
private String [ ] joinOtherStoreNames ;
private String queryableStoreName ;
private StoreBuilder < KeyValueStore < K , VR > > storeBuilder ;
private KTableKTableJoinNodeBuilder ( ) {
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withJoinThisStoreNames ( final String [ ] joinThisStoreNames ) {
this . joinThisStoreNames = joinThisStoreNames ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withNodeName ( final String nodeName ) {
this . nodeName = nodeName ;
return this ;
}
@ -131,59 +159,69 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
@@ -131,59 +159,69 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withNodeName ( final String nodeName ) {
this . nodeName = nodeName ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withJoinOtherProcessorParameters ( final ProcessorParameters < K , Change < V2 > > joinOtherProcessorParameters ) {
this . joinOtherProcessorParameters = joinOtherProcessorParameters ;
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withJoinOtherStoreNames ( final String [ ] joinOtherStoreNames ) {
this . joinOtherStoreNames = joinOtherStoreNames ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withThisJoinSideNodeName ( final String thisJoinSide ) {
this . thisJoinSide = thisJoinSide ;
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withJoinOtherProcessorParameters ( final ProcessorParameters < K , Change < V2 > > joinOtherProcessorParameters ) {
this . joinOtherProcessorParameters = joinOtherProcessorParameters ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withOtherJoinSideNodeName ( final String otherJoinSide ) {
this . otherJoinSide = otherJoinSide ;
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withJoinMergeProcessorParameters ( final ProcessorParameters < K , Change < VR > > joinMergeProcessorParameters ) {
this . joinMergeProcessorParameters = joinMergeProcessorParameters ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withKeySerde ( final Serde < K > keySerde ) {
this . keySerde = keySerde ;
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withValueJoiner ( final ValueJoiner < ? super Change < V1 > , ? super Change < V2 > , ? extends Change < VR > > valueJoiner ) {
this . valueJoiner = valueJoiner ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withValueSerde ( final Serde < VR > valueSerde ) {
this . valueSerde = valueSerde ;
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withThisJoinSideNodeName ( final String thisJoinSide ) {
this . thisJoinSide = thisJoinSide ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withJoinThisStoreNames ( final String [ ] joinThisStoreNames ) {
this . joinThisStoreNames = joinThisStoreNames ;
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withOtherJoinSideNodeName ( final String otherJoinSide ) {
this . otherJoinSide = otherJoinSide ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withJoinOtherStoreNames ( final String [ ] joinOtherStoreNames ) {
this . joinOtherStoreNames = joinOtherStoreNames ;
return this ;
}
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withMaterializedInternal (
final MaterializedInternal < K , VR , KeyValueStore < Bytes , byte [ ] > > materializedInternal ) {
this . materializedInternal = materializedInternal ;
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withQueryableStoreName ( final String queryableStoreName ) {
this . queryableStoreName = queryableStoreName ;
return this ;
}
public KTableKTableJoinNode < K , V1 , V2 , VR > build ( ) {
public KTableKTableJoinNodeBuilder < K , V1 , V2 , VR > withStoreBuilder ( final StoreBuilder < KeyValueStore < K , VR > > storeBuilder ) {
this . storeBuilder = storeBuilder ;
return this ;
}
@SuppressWarnings ( "unchecked" )
public KTableKTableJoinNode < K , V1 , V2 , VR > build ( ) {
return new KTableKTableJoinNode < > ( nodeName ,
valueJoiner ,
joinThisProcessorParameters ,
joinOtherProcessorParameters ,
joinMergeProcessorParameters ,
materializedInternal ,
thisJoinSide ,
otherJoinSide ,
joinThisStoreNames ,
joinOtherStoreNames ) ;
joinThisProcessorParameters ,
joinOtherProcessorParameters ,
new ProcessorParameters < > (
KTableKTableJoinMerger . of (
( KTableProcessorSupplier < K , V1 , VR > ) ( joinThisProcessorParameters . processorSupplier ( ) ) ,
( KTableProcessorSupplier < K , V2 , VR > ) ( joinOtherProcessorParameters . processorSupplier ( ) ) ,
queryableStoreName ) ,
nodeName ) ,
thisJoinSide ,
otherJoinSide ,
keySerde ,
valueSerde ,
joinThisStoreNames ,
joinOtherStoreNames ,
storeBuilder ) ;
}
}
}