@ -30,6 +30,8 @@ import org.apache.kafka.streams.processor.internals.SinkNode;
import org.apache.kafka.streams.processor.internals.SourceNode ;
import org.apache.kafka.streams.processor.internals.SourceNode ;
import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor.SubscriptionUpdates ;
import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor.SubscriptionUpdates ;
import org.apache.kafka.streams.state.internals.RocksDBWindowStoreSupplier ;
import org.apache.kafka.streams.state.internals.RocksDBWindowStoreSupplier ;
import org.slf4j.Logger ;
import org.slf4j.LoggerFactory ;
import java.util.ArrayList ;
import java.util.ArrayList ;
import java.util.Arrays ;
import java.util.Arrays ;
@ -101,6 +103,8 @@ public class TopologyBuilder {
private Map < Integer , Set < String > > nodeGroups = null ;
private Map < Integer , Set < String > > nodeGroups = null ;
private static final Logger log = LoggerFactory . getLogger ( TopologyBuilder . class ) ;
private static class StateStoreFactory {
private static class StateStoreFactory {
public final Set < String > users ;
public final Set < String > users ;
@ -831,14 +835,6 @@ public class TopologyBuilder {
public synchronized Map < Integer , TopicsInfo > topicGroups ( ) {
public synchronized Map < Integer , TopicsInfo > topicGroups ( ) {
Map < Integer , TopicsInfo > topicGroups = new LinkedHashMap < > ( ) ;
Map < Integer , TopicsInfo > topicGroups = new LinkedHashMap < > ( ) ;
if ( subscriptionUpdates . hasUpdates ( ) ) {
for ( Map . Entry < String , Pattern > stringPatternEntry : nodeToSourcePatterns . entrySet ( ) ) {
SourceNodeFactory sourceNode = ( SourceNodeFactory ) nodeFactories . get ( stringPatternEntry . getKey ( ) ) ;
//need to update nodeToSourceTopics with topics matched from given regex
nodeToSourceTopics . put ( stringPatternEntry . getKey ( ) , sourceNode . getTopics ( subscriptionUpdates . getUpdates ( ) ) ) ;
}
}
if ( nodeGroups = = null )
if ( nodeGroups = = null )
nodeGroups = makeNodeGroups ( ) ;
nodeGroups = makeNodeGroups ( ) ;
@ -897,6 +893,17 @@ public class TopologyBuilder {
return Collections . unmodifiableMap ( topicGroups ) ;
return Collections . unmodifiableMap ( topicGroups ) ;
}
}
private void setRegexMatchedTopicsToSourceNodes ( ) {
if ( subscriptionUpdates . hasUpdates ( ) ) {
for ( Map . Entry < String , Pattern > stringPatternEntry : nodeToSourcePatterns . entrySet ( ) ) {
SourceNodeFactory sourceNode = ( SourceNodeFactory ) nodeFactories . get ( stringPatternEntry . getKey ( ) ) ;
//need to update nodeToSourceTopics with topics matched from given regex
nodeToSourceTopics . put ( stringPatternEntry . getKey ( ) , sourceNode . getTopics ( subscriptionUpdates . getUpdates ( ) ) ) ;
log . debug ( "nodeToSourceTopics {}" , nodeToSourceTopics ) ;
}
}
}
private InternalTopicConfig createInternalTopicConfig ( final StateStoreSupplier supplier , final String name ) {
private InternalTopicConfig createInternalTopicConfig ( final StateStoreSupplier supplier , final String name ) {
if ( ! ( supplier instanceof RocksDBWindowStoreSupplier ) ) {
if ( ! ( supplier instanceof RocksDBWindowStoreSupplier ) ) {
return new InternalTopicConfig ( name , Collections . singleton ( InternalTopicConfig . CleanupPolicy . compact ) , supplier . logConfig ( ) ) ;
return new InternalTopicConfig ( name , Collections . singleton ( InternalTopicConfig . CleanupPolicy . compact ) , supplier . logConfig ( ) ) ;
@ -999,7 +1006,9 @@ public class TopologyBuilder {
return this . topicPattern ;
return this . topicPattern ;
}
}
public synchronized void updateSubscriptions ( SubscriptionUpdates subscriptionUpdates ) {
public synchronized void updateSubscriptions ( SubscriptionUpdates subscriptionUpdates , String threadId ) {
log . debug ( "stream-thread [{}] updating builder with {} topic(s) with possible matching regex subscription(s)" , threadId , subscriptionUpdates ) ;
this . subscriptionUpdates = subscriptionUpdates ;
this . subscriptionUpdates = subscriptionUpdates ;
setRegexMatchedTopicsToSourceNodes ( ) ;
}
}
}
}