You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ZhenChun Pan (Jira)" <ji...@apache.org> on 2021/12/24 08:51:00 UTC

[jira] [Comment Edited] (KAFKA-13564) Kafka keep print NOT_LEADER_OR_FOLLOWER in log file after one broker dropped, and the producer can not work.

    [ https://issues.apache.org/jira/browse/KAFKA-13564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17464929#comment-17464929 ] 

ZhenChun Pan edited comment on KAFKA-13564 at 12/24/21, 8:50 AM:
-----------------------------------------------------------------

[~showuon], I add some log imformation about the topic partition ceae-1002-flink-characteristic-instance-data-0 in the description, include the server.log and state-change.log of broker0, and controller.log in broker2.


was (Author: JIRAUSER280457):
[~showuon] I add some log imformation about the topic partition ceae-1002-flink-characteristic-instance-data-0 in the description, include the server.log and state-change.log of broker0, and controller.log in broker2.

> Kafka keep print NOT_LEADER_OR_FOLLOWER in log file after one broker dropped, and the producer can not work.
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-13564
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13564
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.7.0
>            Reporter: ZhenChun Pan
>            Priority: Major
>
> The machine of broker0 dropped, and some partition change the leader to broker1. We can found message like below in state-change.log:
> [2021-12-11 15:34:14,868] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=3, leader=1, leaderEpoch=8, isr=[1], zkVersion*{*}{{*}}{{*}}{{*}}#{*}#{{*}}{{*}}{*}**, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 3 with correlation id 0 (state.change.logger)
> But we found server.log keep print logs like below:
> [2021-12-11 15:34:30,272] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
> And the producer also can not work and the client print messages below:
> [2021-12-11 16:00:00,703] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=4] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
> We begin to resume broker0 at the time 2021-12-11 15:33:10, but did not work. So we restart all brokers of kafka cluster at the time 2021-12-11 16:32:31, and fix the trouble.
> The log of ceae-1002-flink-characteristic-instance-data-0 below:
> Search"ceae-1002-flink-characteristic-instance-data-0"
>   \20211211\broker0\server.log
>     Line 40502: [2021-12-11 15:34:00,283] INFO [Log partition=ceae-1002-flink-characteristic-instance-data-0, dir=/opt/kafka-service/data] Recovering unflushed segment 0 (kafka.log.Log)
>     Line 40503: [2021-12-11 15:34:00,283] INFO [Log partition=ceae-1002-flink-characteristic-instance-data-0, dir=/opt/kafka-service/data] Loading producer state till offset 0 with message format version***{*}{{*}}#{*}#{*}**** (kafka.log.Log)
>     Line 40570: [2021-12-11 15:34:00,542] INFO [ProducerStateManager partition=ceae-1002-flink-characteristic-instance-data-0] Writing producer snapshot at offset 82323 (kafka.log.ProducerStateManager)
>     Line 40577: [2021-12-11 15:34:00,566] INFO [Log partition=ceae-1002-flink-characteristic-instance-data-0, dir=/opt/kafka-service/data] Loading producer state till offset 82323 with message format version***{*}{{*}}#{*}#{*}**** (kafka.log.Log)
>     Line 40578: [2021-12-11 15:34:00,567] INFO [ProducerStateManager partition=ceae-1002-flink-characteristic-instance-data-0] Loading producer state from snapshot file '/opt/kafka-service/data/ceae-1002-flink-characteristic-instance-data-0/00000000000000082323.snapshot' (kafka.log.ProducerStateManager)
>     Line 40578: [2021-12-11 15:34:00,567] INFO [ProducerStateManager partition=ceae-1002-flink-characteristic-instance-data-0] Loading producer state from snapshot file '/opt/kafka-service/data/ceae-1002-flink-characteristic-instance-data-0/00000000000000082323.snapshot' (kafka.log.ProducerStateManager)
>     Line 40581: [2021-12-11 15:34:00,568] INFO Completed load of Log(dir=/opt/kafka-service/data/ceae-1002-flink-characteristic-instance-data-0, topic=ceae-1002-flink-characteristic-instance-data, partition=0, highWatermark=0, lastStableOffset=0, logStartOffset=0, logEndOffset=82323) with 1 segments in 287ms (437/487 loaded in /opt/kafka-service/data) (kafka.log.LogManager)
>     Line 41109: [2021-12-11 15:34:17,641] INFO [Partition ceae-1002-flink-characteristic-instance-data-0 broker=0] Log loaded for partition ceae-1002-flink-characteristic-instance-data-0 with initial high watermark 82323 (kafka.cluster.Partition)
>     Line 41109: [2021-12-11 15:34:17,641] INFO [Partition ceae-1002-flink-characteristic-instance-data-0 broker=0] Log loaded for partition ceae-1002-flink-characteristic-instance-data-0 with initial high watermark 82323 (kafka.cluster.Partition)
>     Line 41832: [2021-12-11 15:34:26,032] INFO [ReplicaFetcherManager on broker 0] Added fetcher to broker 1 for partitions HashMap(ceae-1002-flink-characteristic-instance-data-0 -> (offset=82323, leaderEpoch=8), topic-1002-sia-analysed-alarms-8 -> (offset=0, leaderEpoch=7), cea-1002-event-output-0 -> (offset=0, leaderEpoch=7), ceae-1002-flink-suppression-message-9 -> (offset=0, leaderEpoch=7)) (kafka.server.ReplicaFetcherManager)
>     Line 41848: [2021-12-11 15:34:26,112] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 42034: [2021-12-11 15:34:27,171] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 42152: [2021-12-11 15:34:28,201] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 42305: [2021-12-11 15:34:29,212] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
> .......................
>     Line 122789: [2021-12-11 16:30:01,126] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 122813: [2021-12-11 16:30:02,135] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 122839: [2021-12-11 16:30:03,152] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 125359: [2021-12-11 16:31:48,031] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 125383: [2021-12-11 16:31:49,036] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 125408: [2021-12-11 16:31:50,045] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 125432: [2021-12-11 16:31:51,062] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: NOT_LEADER_OR_FOLLOWER (kafka.server.ReplicaFetcherThread)
>     Line 127446: [2021-12-11 16:32:45,251] INFO [Log partition=ceae-1002-flink-characteristic-instance-data-0, dir=/opt/kafka-service/data] Recovering unflushed segment 0 (kafka.log.Log)
>     Line 127447: [2021-12-11 16:32:45,252] INFO [Log partition=ceae-1002-flink-characteristic-instance-data-0, dir=/opt/kafka-service/data] Loading producer state till offset 0 with message format version***{*}{{*}}#{*}#{*}**** (kafka.log.Log)
>     Line 127517: [2021-12-11 16:32:45,405] INFO [ProducerStateManager partition=ceae-1002-flink-characteristic-instance-data-0] Writing producer snapshot at offset 82323 (kafka.log.ProducerStateManager)
>     Line 127518: [2021-12-11 16:32:45,414] INFO [Log partition=ceae-1002-flink-characteristic-instance-data-0, dir=/opt/kafka-service/data] Loading producer state till offset 82323 with message format version***{*}{{*}}#{*}#{*}**** (kafka.log.Log)
>     Line 127519: [2021-12-11 16:32:45,415] INFO [ProducerStateManager partition=ceae-1002-flink-characteristic-instance-data-0] Loading producer state from snapshot file '/opt/kafka-service/data/ceae-1002-flink-characteristic-instance-data-0/00000000000000082323.snapshot' (kafka.log.ProducerStateManager)
>     Line 127519: [2021-12-11 16:32:45,415] INFO [ProducerStateManager partition=ceae-1002-flink-characteristic-instance-data-0] Loading producer state from snapshot file '/opt/kafka-service/data/ceae-1002-flink-characteristic-instance-data-0/00000000000000082323.snapshot' (kafka.log.ProducerStateManager)
>     Line 127520: [2021-12-11 16:32:45,416] INFO Completed load of Log(dir=/opt/kafka-service/data/ceae-1002-flink-characteristic-instance-data-0, topic=ceae-1002-flink-characteristic-instance-data, partition=0, highWatermark=0, lastStableOffset=0, logStartOffset=0, logEndOffset=82323) with 1 segments in 166ms (437/487 loaded in /opt/kafka-service/data) (kafka.log.LogManager)
>     Line 128177: [2021-12-11 16:32:59,045] INFO [Partition ceae-1002-flink-characteristic-instance-data-0 broker=0] Log loaded for partition ceae-1002-flink-characteristic-instance-data-0 with initial high watermark 82323 (kafka.cluster.Partition)
>     Line 128177: [2021-12-11 16:32:59,045] INFO [Partition ceae-1002-flink-characteristic-instance-data-0 broker=0] Log loaded for partition ceae-1002-flink-characteristic-instance-data-0 with initial high watermark 82323 (kafka.cluster.Partition)
>     Line 129436: [2021-12-11 16:33:02,451] INFO [ReplicaFetcherManager on broker 0] Added fetcher to broker 1 for partitions HashMap(ceae-1002-flink-characteristic-instance-data-0 -> (offset=82323, leaderEpoch=10), topic-1002-sia-analysed-alarms-8 -> (offset=0, leaderEpoch=10), cea-1002-event-output-0 -> (offset=0, leaderEpoch=10), ceae-1002-flink-suppression-message-9 -> (offset=0, leaderEpoch=10)) (kafka.server.ReplicaFetcherManager)
>     Line 129470: [2021-12-11 16:33:02,535] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: UNKNOWN_LEADER_EPOCH (kafka.server.ReplicaFetcherThread)
>     Line 129754: [2021-12-11 16:33:03,762] INFO [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=6] Retrying leaderEpoch request for partition ceae-1002-flink-characteristic-instance-data-0 as the leader reported an error: UNKNOWN_LEADER_EPOCH (kafka.server.ReplicaFetcherThread)
>     Line 129941: [2021-12-11 16:33:04,834] INFO [Log partition=ceae-1002-flink-characteristic-instance-data-0, dir=/opt/kafka-service/data] Truncating to 82323 has no effect as the largest offset in the log is 82322 (kafka.log.Log)
> Search "ceae-1002-flink-characteristic-instance-data-0"
>   20211211\broker0\state-change.log
>     Line 272: [2021-12-11 15:34:14,868] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=3, leader=1, leaderEpoch=8, isr=[1], zkVersion***{*}{{*}}#{*}#{*}****, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 3 with correlation id 0 (state.change.logger)
>     Line 1438: [2021-12-11 15:34:17,025] TRACE [Broker id=0] Handling LeaderAndIsr request correlationId 1 from controller 2 epoch 3 starting the become-follower transition for partition ceae-1002-flink-characteristic-instance-data-0 with leader 1 (state.change.logger)
>     Line 1925: [2021-12-11 15:34:17,642] INFO [Broker id=0] Follower ceae-1002-flink-characteristic-instance-data-0 starts at leader epoch 8 from offset 82323 with high watermark 82323. Previous leader epoch was -1. (state.change.logger)
>     Line 2413: [2021-12-11 15:34:30,448] TRACE [Broker id=0] Completed LeaderAndIsr request correlationId 1 from controller 2 epoch 3 for the become-follower transition for partition ceae-1002-flink-characteristic-instance-data-0 with leader 1 (state.change.logger)
>     Line 3301: [2021-12-11 15:34:31,584] INFO [Broker id=0] Ignoring LeaderAndIsr request from controller 2 with correlation id 2 epoch 3 for partition ceae-1002-flink-characteristic-instance-data-0 since its associated leader epoch 8 matches the current leader epoch (state.change.logger)
>     Line 3793: [2021-12-11 15:34:32,343] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=3, leader=1, leaderEpoch=8, isr=[1], zkVersion***{*}{{*}}#{*}#{*}****, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 3 with correlation id 3 (state.change.logger)
>     Line 5470: [2021-12-11 16:32:57,544] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=4, leader=-1, leaderEpoch=9, isr=[1], zkVersion***{*}{{*}}#{*}#{*}****, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 4 with correlation id 0 (state.change.logger)
>     Line 6636: [2021-12-11 16:32:58,712] TRACE [Broker id=0] Handling LeaderAndIsr request correlationId 1 from controller 2 epoch 4 starting the become-follower transition for partition ceae-1002-flink-characteristic-instance-data-0 with leader -1 (state.change.logger)
>     Line 7123: [2021-12-11 16:32:59,045] ERROR [Broker id=0] Received LeaderAndIsrRequest with correlation id 1 from controller 2 epoch 4 for partition ceae-1002-flink-characteristic-instance-data-0 (last update controller epoch 4) but cannot become follower since the new leader -1 is unavailable. (state.change.logger)
>     Line 7611: [2021-12-11 16:33:01,061] TRACE [Broker id=0] Completed LeaderAndIsr request correlationId 1 from controller 2 epoch 4 for the become-follower transition for partition ceae-1002-flink-characteristic-instance-data-0 with leader -1 (state.change.logger)
>     Line 8103: [2021-12-11 16:33:01,403] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=4, leader=-1, leaderEpoch=9, isr=[1], zkVersion***{*}{{*}}#{*}#{*}****, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 4 with correlation id 2 (state.change.logger)
>     Line 8737: [2021-12-11 16:33:01,758] TRACE [Broker id=0] Handling LeaderAndIsr request correlationId 3 from controller 2 epoch 4 starting the become-follower transition for partition ceae-1002-flink-characteristic-instance-data-0 with leader 1 (state.change.logger)
>     Line 8930: [2021-12-11 16:33:01,839] INFO [Broker id=0] Follower ceae-1002-flink-characteristic-instance-data-0 starts at leader epoch 10 from offset 82323 with high watermark 82323. Previous leader epoch was -1. (state.change.logger)
>     Line 9124: [2021-12-11 16:33:02,974] TRACE [Broker id=0] Completed LeaderAndIsr request correlationId 3 from controller 2 epoch 4 for the become-follower transition for partition ceae-1002-flink-characteristic-instance-data-0 with leader 1 (state.change.logger)
>     Line 9384: [2021-12-11 16:33:03,133] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=4, leader=1, leaderEpoch=10, isr=[1], zkVersion***{*}{{*}}#{*}#{*}****, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 4 with correlation id 4 (state.change.logger)
>     Line 10534: [2021-12-11 16:33:05,471] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=4, leader=1, leaderEpoch=10, isr=[1, 0], zkVersion***{*}{{*}}#{*}#{*}****, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 4 with correlation id 49 (state.change.logger)
>     Line 10945: [2021-12-11 16:38:02,874] TRACE [Broker id=0] Handling LeaderAndIsr request correlationId 74 from controller 2 epoch 4 starting the become-leader transition for partition ceae-1002-flink-characteristic-instance-data-0 (state.change.logger)
>     Line 11126: [2021-12-11 16:38:03,295] INFO [Broker id=0] Leader ceae-1002-flink-characteristic-instance-data-0 starts at leader epoch 11 from offset 82593 with high watermark 82593 ISR [1,0] addingReplicas [] removingReplicas []. Previous leader epoch was 10. (state.change.logger)
>     Line 11306: [2021-12-11 16:38:04,265] TRACE [Broker id=0] Completed LeaderAndIsr request correlationId 74 from controller 2 epoch 4 for the become-leader transition for partition ceae-1002-flink-characteristic-instance-data-0 (state.change.logger)
>     Line 11500: [2021-12-11 16:38:04,565] TRACE [Broker id=0] Cached leader info UpdateMetadataPartitionState(topicName='ceae-1002-flink-characteristic-instance-data', partitionIndex=0, controllerEpoch=4, leader=0, leaderEpoch=11, isr=[1, 0], zkVersion***{*}{{*}}#{*}#{*}****, offlineReplicas=[]) for partition ceae-1002-flink-characteristic-instance-data-0 in response to UpdateMetadata request sent by controller 2 epoch 4 with correlation id 75 (state.change.logger)
> Search "ceae-1002-flink-characteristic-instance-data-0"
>   20211211\broker2\controller.log
>     Line 889: [2021-12-11 15:41:31,186] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 898: [2021-12-11 15:46:31,192] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 907: [2021-12-11 15:51:31,201] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 916: [2021-12-11 15:56:31,207] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 925: [2021-12-11 16:01:31,212] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 934: [2021-12-11 16:06:31,219] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 943: [2021-12-11 16:11:31,227] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 952: [2021-12-11 16:16:31,235] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 961: [2021-12-11 16:21:31,240] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 970: [2021-12-11 16:26:31,247] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 979: [2021-12-11 16:31:31,252] DEBUG [Controller id=2] Topics not in preferred replica for broker 0 HashMap(topic-datasource-access-NetcareLiteKPI-1002-2 -> List(0, 1), IES.tunnel.object.1002-4 -> List(0, 1), cea-1002-event-base-10 -> List(0, 1), Notify_IES_Common_ServiceInstance-3 -> List(0, 1), topic-1002-flink-base-rule-2 -> List(0, 1), IES.tunnel.eventext.1002-4 -> List(0, 1), Notify_IES_Common_ServiceInstance-9 -> List(0, 1), SID_TroubeEvent-7 -> List(0, 1), IES.tunnel.config.1002-3 -> List(0, 1), ceae-1002-event-overload-1 -> List(0, 1), IES.tunnel.resource.1002-11 -> List(0, 2), cea-1002-event-nbi-10 -> List(0, 1), topic-sia-affected-services-6 -> List(0, 1), topic-1002-flink-config-2 -> List(0, 1), IES.tunnel.dts.1002-2 -> List(0, 1), IES.tunnel.syncevent.1002-2 -> List(0, 1), __consumer_offsets-9 -> List(0, 2, 1), topic-1002-sia-service-outer-10 -> List(0, 1), ceae-1002-event-base-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-0 -> List(0, 1), ceae-1002-characteristic-instance-7 -> List(0, 1), topic-1002-flink-original-7 -> List(0, 1), topic-1002-sia-filtered-alarms-0 -> List(0, 1), mateinfo.1002.nfsalarmtopic-6 -> List(0, 1), topic-1002-sia-analysed-alarms-4 -> List(0, 1), IES.tunnel.rmconsumerservice.1002-7 -> List(0, 1), dispatch_alarm_topic-8 -> List(0, 1), cea-1002-config-model-7 -> List(0, 1), topic-1002-sia-filtered-alarms-6 -> List(0, 1), topic-sia-affected-services-9 -> List(0, 2), cea-1002-event-nbi-4 -> List(0, 1), IES.tunnel.alarm.1002-9 -> List(0, 1), IES.tunnel.config.1002-9 -> List(0, 1), topic-1002-flink-original-1 -> List(0, 1), topic-sia-filtered-alarms-8 -> List(0, 1), topic-1002-flink-base-rule-8 -> List(0, 1), topic-1002-sia-analysed-alarms-10 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-6 -> List(0, 1), IES.tunnel.cooperationResource.1002-6 -> List(0, 1), Notify_IES_Common_Customer-8 -> List(0, 1), IES.tunnel.resource.1002-2 -> List(0, 1), topic-1002-flink-reflow-alarm-11 -> List(0, 1), ceae-1002-flink-characteristic-instance-data-0 -> List(0, 1), IE ...
>     Line 1801: [2021-12-11 16:33:05,074] DEBUG [Controller id=2] Updating ISRs for partitions: Set(topic-datasource-access-NetcareLiteKPI-1002-3, cea-1002-event-base-0, IES.tunnel.eventext.1002-6, IES.tunnel.alarm.1002-9, ceae-1002-event-base-10, cea-1002-config-model-7, IES.tunnel.motypeevent-7, IES.tunnel.syncevent.1002-10, ceae-1002-event-output-1, IES.tunnel.dts.1002-8, dispatch_alarm_topic-8, ceae-1002-flink-characteristic-instance-data-0, dispatch_alarm_topic-3, ceae-1002-event-output-5, IES.tunnel.syncevent.1002-2, ceae-1002-flink-suppression-message-9, Notify_IES_Common_Customer-8). (kafka.controller.KafkaController)
>     Line 1812: [2021-12-11 16:33:05,118] DEBUG [Controller id=2] ISR for partition ceae-1002-flink-characteristic-instance-data-0 updated to [1,0] and zkVersion***{*}{{*}}#{*}#{*}**** (kafka.controller.KafkaController)
>     Line 2068: [2021-12-11 16:38:02,885] INFO [Controller id=2] Partition ceae-1002-flink-characteristic-instance-data-0 completed preferred replica leader election. New leader is 0 (kafka.controller.KafkaController)



--
This message was sent by Atlassian Jira
(v8.20.1#820001)