@ -17,15 +17,21 @@
package org.apache.kafka.streams.kstream.internals.graph ;
package org.apache.kafka.streams.kstream.internals.graph ;
import org.apache.kafka.common.serialization.Serdes ;
import org.apache.kafka.streams.KeyValue ;
import org.apache.kafka.streams.StreamsBuilder ;
import org.apache.kafka.streams.StreamsBuilder ;
import org.apache.kafka.streams.StreamsConfig ;
import org.apache.kafka.streams.StreamsConfig ;
import org.apache.kafka.streams.Topology ;
import org.apache.kafka.streams.Topology ;
import org.apache.kafka.streams.kstream.Grouped ;
import org.apache.kafka.streams.kstream.JoinWindows ;
import org.apache.kafka.streams.kstream.JoinWindows ;
import org.apache.kafka.streams.kstream.KStream ;
import org.apache.kafka.streams.kstream.KStream ;
import org.apache.kafka.streams.kstream.Materialized ;
import org.apache.kafka.streams.kstream.Produced ;
import org.apache.kafka.streams.kstream.TimeWindows ;
import org.apache.kafka.streams.kstream.TimeWindows ;
import org.apache.kafka.streams.kstream.ValueJoiner ;
import org.apache.kafka.streams.kstream.ValueJoiner ;
import org.junit.Test ;
import org.junit.Test ;
import java.time.Duration ;
import java.util.ArrayList ;
import java.util.ArrayList ;
import java.util.Arrays ;
import java.util.Arrays ;
import java.util.List ;
import java.util.List ;
@ -67,6 +73,33 @@ public class StreamsGraphTest {
}
}
@Test
public void shouldBeAbleToProcessNestedMultipleKeyChangingNodes ( ) {
final Properties properties = new Properties ( ) ;
properties . setProperty ( StreamsConfig . APPLICATION_ID_CONFIG , "test-application" ) ;
properties . setProperty ( StreamsConfig . BOOTSTRAP_SERVERS_CONFIG , "localhost:9092" ) ;
properties . setProperty ( StreamsConfig . TOPOLOGY_OPTIMIZATION , StreamsConfig . OPTIMIZE ) ;
final StreamsBuilder builder = new StreamsBuilder ( ) ;
final KStream < String , String > inputStream = builder . stream ( "inputTopic" ) ;
final KStream < String , String > changedKeyStream = inputStream . selectKey ( ( k , v ) - > v . substring ( 0 , 5 ) ) ;
// first repartition
changedKeyStream . groupByKey ( Grouped . as ( "count-repartition" ) )
. count ( Materialized . as ( "count-store" ) )
. toStream ( ) . to ( "count-topic" , Produced . with ( Serdes . String ( ) , Serdes . Long ( ) ) ) ;
// second repartition
changedKeyStream . groupByKey ( Grouped . as ( "windowed-repartition" ) )
. windowedBy ( TimeWindows . of ( Duration . ofSeconds ( 5 ) ) )
. count ( Materialized . as ( "windowed-count-store" ) )
. toStream ( )
. map ( ( k , v ) - > KeyValue . pair ( k . key ( ) , v ) ) . to ( "windowed-count" , Produced . with ( Serdes . String ( ) , Serdes . Long ( ) ) ) ;
builder . build ( properties ) ;
}
@Test
@Test
public void shouldNotOptimizeWithValueOrKeyChangingOperatorsAfterInitialKeyChange ( ) {
public void shouldNotOptimizeWithValueOrKeyChangingOperatorsAfterInitialKeyChange ( ) {