@ -36,6 +36,9 @@ import org.apache.kafka.streams.StreamsConfig;
@@ -36,6 +36,9 @@ import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsMetrics ;
import org.apache.kafka.streams.kstream.KStreamBuilder ;
import org.apache.kafka.streams.processor.TaskId ;
import org.apache.kafka.streams.processor.TopologyBuilder ;
import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo ;
import org.apache.kafka.streams.state.HostInfo ;
import org.apache.kafka.streams.state.Stores ;
import org.apache.kafka.test.MockClientSupplier ;
import org.apache.kafka.test.MockInternalTopicManager ;
@ -52,6 +55,7 @@ import java.io.File;
@@ -52,6 +55,7 @@ import java.io.File;
import java.lang.reflect.Field ;
import java.nio.ByteBuffer ;
import java.nio.file.Files ;
import java.util.ArrayList ;
import java.util.Arrays ;
import java.util.Collection ;
import java.util.Collections ;
@ -1499,6 +1503,93 @@ public class StreamThreadTest {
@@ -1499,6 +1503,93 @@ public class StreamThreadTest {
assertFalse ( testStreamTask . committed ) ;
}
@Test
@SuppressWarnings ( "unchecked" )
public void shouldAlwaysUpdateWithLatestTopicsFromStreamPartitionAssignor ( ) throws Exception {
final TopologyBuilder topologyBuilder = new TopologyBuilder ( ) ;
topologyBuilder . addSource ( "source" , Pattern . compile ( "t.*" ) ) ;
topologyBuilder . addProcessor ( "processor" , new MockProcessorSupplier ( ) , "source" ) ;
final StreamThread thread = new StreamThread (
topologyBuilder ,
config ,
clientSupplier ,
applicationId ,
clientId ,
processId ,
metrics ,
mockTime ,
new StreamsMetadataState ( topologyBuilder , StreamsMetadataState . UNKNOWN_HOST ) ,
0 ) ;
final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor ( ) ;
final Map < String , Object > configurationMap = new HashMap < > ( ) ;
configurationMap . put ( StreamsConfig . InternalConfig . STREAM_THREAD_INSTANCE , thread ) ;
configurationMap . put ( StreamsConfig . NUM_STANDBY_REPLICAS_CONFIG , 0 ) ;
partitionAssignor . configure ( configurationMap ) ;
thread . setPartitionAssignor ( partitionAssignor ) ;
final Field
nodeToSourceTopicsField =
topologyBuilder . getClass ( ) . getDeclaredField ( "nodeToSourceTopics" ) ;
nodeToSourceTopicsField . setAccessible ( true ) ;
final Map < String , List < String > >
nodeToSourceTopics =
( Map < String , List < String > > ) nodeToSourceTopicsField . get ( topologyBuilder ) ;
final List < TopicPartition > topicPartitions = new ArrayList < > ( ) ;
final TopicPartition topicPartition1 = new TopicPartition ( "topic-1" , 0 ) ;
final TopicPartition topicPartition2 = new TopicPartition ( "topic-2" , 0 ) ;
final TopicPartition topicPartition3 = new TopicPartition ( "topic-3" , 0 ) ;
final TaskId taskId1 = new TaskId ( 0 , 0 ) ;
final TaskId taskId2 = new TaskId ( 0 , 0 ) ;
final TaskId taskId3 = new TaskId ( 0 , 0 ) ;
List < TaskId > activeTasks = Arrays . asList ( taskId1 ) ;
final Map < TaskId , Set < TopicPartition > > standbyTasks = new HashMap < > ( ) ;
AssignmentInfo info = new AssignmentInfo ( activeTasks , standbyTasks , new HashMap < HostInfo , Set < TopicPartition > > ( ) ) ;
topicPartitions . addAll ( Arrays . asList ( topicPartition1 ) ) ;
PartitionAssignor . Assignment assignment = new PartitionAssignor . Assignment ( topicPartitions , info . encode ( ) ) ;
partitionAssignor . onAssignment ( assignment ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . size ( ) = = 1 ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . contains ( "topic-1" ) ) ;
topicPartitions . clear ( ) ;
activeTasks = Arrays . asList ( taskId1 , taskId2 ) ;
info = new AssignmentInfo ( activeTasks , standbyTasks , new HashMap < HostInfo , Set < TopicPartition > > ( ) ) ;
topicPartitions . addAll ( Arrays . asList ( topicPartition1 , topicPartition2 ) ) ;
assignment = new PartitionAssignor . Assignment ( topicPartitions , info . encode ( ) ) ;
partitionAssignor . onAssignment ( assignment ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . size ( ) = = 2 ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . contains ( "topic-1" ) ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . contains ( "topic-2" ) ) ;
topicPartitions . clear ( ) ;
activeTasks = Arrays . asList ( taskId1 , taskId2 , taskId3 ) ;
info = new AssignmentInfo ( activeTasks , standbyTasks ,
new HashMap < HostInfo , Set < TopicPartition > > ( ) ) ;
topicPartitions . addAll ( Arrays . asList ( topicPartition1 , topicPartition2 , topicPartition3 ) ) ;
assignment = new PartitionAssignor . Assignment ( topicPartitions , info . encode ( ) ) ;
partitionAssignor . onAssignment ( assignment ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . size ( ) = = 3 ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . contains ( "topic-1" ) ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . contains ( "topic-2" ) ) ;
assertTrue ( nodeToSourceTopics . get ( "source" ) . contains ( "topic-3" ) ) ;
}
private void initPartitionGrouper ( final StreamsConfig config ,
final StreamThread thread ,
final MockClientSupplier clientSupplier ) {