You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Raman Gupta (Jira)" <ji...@apache.org> on 2019/10/01 17:39:00 UTC

[jira] [Commented] (KAFKA-8966) Stream state does not transition to RUNNING on client, broker consumer group shows RUNNING

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

Raman Gupta commented on KAFKA-8966:
------------------------------------

The only thing I did that *might* be related to this is that I updated the replication factor of all the streams input and internal topics to 3, and then ran a `kafka-reassign-partitions` command to rebalance the partitions across the 4 brokers. However, I confirmed the reassignment was successful and all topic partitions were in-sync and had the required ISRs.

The stream was also restarted (but not reset) more than once after the repartitioning.

> Stream state does not transition to RUNNING on client, broker consumer group shows RUNNING
> ------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-8966
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8966
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 2.3.0
>            Reporter: Raman Gupta
>            Priority: Critical
>
> I have a Kafka stream that has been running fine until recently. The new behavior I see is that the stream state on the client goes from CREATED to REBALANCING, but never transitions from REBALANCING to RUNNING.
> However, at the same time, if I look at the offsets of the corresponding consumer group, the consumer group appears to be consuming from the topic and has no lag. And yet, the client never made a state change to RUNNING. This is confirmed by calling `streams.close` on the stream and noting the state change goes from REBALANCING to PENDING_SHUTDOWN instead of RUNNING to PENDING_SHUTDOWN as expected.
> I use the state change to enable queries on the stream store -- if the state change listener never triggers to the RUNNING state, there is no way to know when the client is available for queries.
> Yes, I have confirmed its the correct consumer group. Yes, the consumer group has no consumers when I shut down the client stream.
> Server logs:
> kafka-2 kafka 2019-10-01T17:05:06.284647485Z [2019-10-01 17:05:06,284] INFO [GroupCoordinator 2]: Preparing to rebalance group arena-rg-uiService-fileStatusStore-stream in state PreparingRebalance with old generation 4 (__consumer_offsets-42) (reason: Adding new member arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer-10d56cf1-1901-41f8-bcb9-99a892231dfe with group instanceid None) (kafka.coordinator.group.GroupCoordinator)
> kafka-2 kafka 2019-10-01T17:05:36.285016762Z [2019-10-01 17:05:36,284] INFO [GroupCoordinator 2]: Stabilized group arena-rg-uiService-fileStatusStore-stream generation 5 (__consumer_offsets-42) (kafka.coordinator.group.GroupCoordinator)
> kafka-2 kafka 2019-10-01T17:05:36.733765635Z [2019-10-01 17:05:36,733] INFO [GroupCoordinator 2]: Assignment received from leader for group arena-rg-uiService-fileStatusStore-stream for generation 5 (kafka.coordinator.group.GroupCoordinator)
> Client logs:
> 2019-10-01 13:05:05,031 INFO  --- [                main] org.apa.kaf.str.StreamsConfig                     : StreamsConfig values: 
> 	application.id = arena-rg-uiService-fileStatusStore-stream
> 	application.server = 
> 	bootstrap.servers = [kafka-0.k8s-dev.azure.xxx.com:9093, kafka-1.k8s-dev.azure.xxx.com:9093]
> 	buffered.records.per.partition = 1000
> 	cache.max.bytes.buffering = 10485760
> 	client.id = 
> 	commit.interval.ms = 30000
> 	connections.max.idle.ms = 540000
> 	default.deserialization.exception.handler = class org.apache.kafka.streams.errors.LogAndFailExceptionHandler
> 	default.key.serde = class io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde
> 	default.production.exception.handler = class org.apache.kafka.streams.errors.DefaultProductionExceptionHandler
> 	default.timestamp.extractor = class org.apache.kafka.streams.processor.FailOnInvalidTimestamp
> 	default.value.serde = class io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde
> 	max.task.idle.ms = 0
> 	metadata.max.age.ms = 300000
> 	metric.reporters = []
> 	metrics.num.samples = 2
> 	metrics.recording.level = INFO
> 	metrics.sample.window.ms = 30000
> 	num.standby.replicas = 0
> 	num.stream.threads = 1
> 	partition.grouper = class org.apache.kafka.streams.processor.DefaultPartitionGrouper
> 	poll.ms = 100
> 	processing.guarantee = at_least_once
> 	receive.buffer.bytes = 32768
> 	reconnect.backoff.max.ms = 1000
> 	reconnect.backoff.ms = 50
> 	replication.factor = 3
> 	request.timeout.ms = 40000
> 	retries = 0
> 	retry.backoff.ms = 100
> 	rocksdb.config.setter = null
> 	security.protocol = PLAINTEXT
> 	send.buffer.bytes = 131072
> 	state.cleanup.delay.ms = 600000
> 	state.dir = /tmp/kafka-streams
> 	topology.optimization = none
> 	upgrade.from = null
> 	windowstore.changelog.additional.retention.ms = 86400000
> 2019-10-01 13:05:05,090 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka version: 2.3.0
> 2019-10-01 13:05:05,091 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka commitId: fc1aaa116b661c8a
> 2019-10-01 13:05:05,091 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka startTimeMs: 1569949505090
> 2019-10-01 13:05:05,092 INFO  --- [                main] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] Creating restore consumer client
> 2019-10-01 13:05:05,123 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka version: 2.3.0
> 2019-10-01 13:05:05,123 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka commitId: fc1aaa116b661c8a
> 2019-10-01 13:05:05,123 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka startTimeMs: 1569949505123
> 2019-10-01 13:05:05,137 INFO  --- [                main] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] Creating shared producer client
> 2019-10-01 13:05:05,148 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka version: 2.3.0
> 2019-10-01 13:05:05,149 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka commitId: fc1aaa116b661c8a
> 2019-10-01 13:05:05,149 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka startTimeMs: 1569949505148
> 2019-10-01 13:05:05,169 INFO  --- [                main] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] Creating consumer client
> 2019-10-01 13:05:05,194 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka version: 2.3.0
> 2019-10-01 13:05:05,194 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka commitId: fc1aaa116b661c8a
> 2019-10-01 13:05:05,194 INFO  --- [                main] org.apa.kaf.com.uti.AppInfoParser                 : Kafka startTimeMs: 1569949505194
> 2019-10-01 13:05:05,200 INFO  --- [                main] org.apa.kaf.str.KafkaStreams                      : stream-client [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6] State transition from CREATED to REBALANCING
> 2019-10-01 13:05:05,201 INFO  --- [                main] com.xxx.mic.kaf.AbstractStreamProcessor           : Kafka stream FileStatusStore state change CREATED -> REBALANCING
> 2019-10-01 13:05:05,202 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] Starting
> 2019-10-01 13:05:05,202 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] State transition from CREATED to STARTING
> 2019-10-01 13:05:05,203 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.KafkaConsumer                 : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Subscribed to pattern: 'arena-rg-dlq-reporting-events|arena-rg-file-events-v3|arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition|arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition|arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition'
> 2019-10-01 13:05:05,302 DEBUG --- [                main] m.red.mic.boo.StopSignalHandlerLifecycleController: Stop signal handler installed
> 2019-10-01 13:05:05,304 DEBUG --- [                main] com.xxx.mic.boo.Bootable                          : Started application in 0.359 seconds.
> 2019-10-01 13:05:05,304 INFO  --- [                main] com.xxx.mic.boo.Bootable                          : => Running
> 2019-10-01 13:05:05,315 INFO  --- [eamThread-1-producer] org.apa.kaf.cli.Metadata                          : [Producer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-producer] Cluster ID: ZZ2bYOBRRDCV-nCSBUzyVQ
> 2019-10-01 13:05:05,738 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.Metadata                          : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Cluster ID: ZZ2bYOBRRDCV-nCSBUzyVQ
> 2019-10-01 13:05:05,770 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.AbstractCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Discovered group coordinator kafka-2.k8s-dev.azure.xxx.com:9093 (id: 2147483645 rack: null)
> 2019-10-01 13:05:06,227 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Revoking previously assigned partitions []
> 2019-10-01 13:05:06,227 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] State transition from STARTING to PARTITIONS_REVOKED
> 2019-10-01 13:05:06,227 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.KafkaConsumer                 : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-restore-consumer, groupId=null] Unsubscribed all topics or patterns and assigned partitions
> 2019-10-01 13:05:06,228 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] partition revocation took 1 ms.
> 	suspended active tasks: []
> 	suspended standby tasks: []
> 2019-10-01 13:05:06,228 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.AbstractCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] (Re-)joining group
> 2019-10-01 13:05:06,266 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.AbstractCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] (Re-)joining group
> 2019-10-01 13:05:36,658 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.str.pro.int.StreamsPartitionAssignor  : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer] Assigned tasks to clients as {ca9602d3-a249-4f1e-af1d-5c80246d9dd6=[activeTasks: ([0_0, 0_1, 1_0, 0_2, 1_1, 2_0, 0_3, 1_2, 2_1, 3_0, 0_4, 2_2, 3_1, 0_5, 2_3, 3_2, 0_6, 2_4, 3_3, 0_7, 2_5, 3_4, 0_8, 2_6, 3_5, 0_9, 2_7, 3_6, 0_10, 2_8, 3_7, 0_11, 2_9, 3_8, 0_12, 2_10, 3_9, 0_13, 2_11, 3_10, 0_14, 2_12, 3_11, 0_15, 2_13, 3_12, 0_16, 2_14, 3_13, 0_17, 2_15, 3_14, 0_18, 2_16, 3_15, 0_19, 2_17, 3_16, 0_20, 2_18, 3_17, 0_21, 2_19, 3_18, 0_22, 2_20, 3_19, 0_23, 2_21, 3_20, 0_24, 2_22, 3_21, 0_25, 2_23, 3_22, 0_26, 2_24, 3_23, 0_27, 2_25, 3_24, 0_28, 2_26, 3_25, 0_29, 2_27, 3_26, 0_30, 2_28, 3_27, 0_31, 2_29, 3_28, 0_32, 2_30, 3_29, 0_33, 2_31, 3_30, 0_34, 2_32, 3_31, 0_35, 2_33, 3_32, 0_36, 2_34, 3_33, 0_37, 2_35, 3_34, 0_38, 2_36, 3_35, 0_39, 2_37, 3_36, 0_40, 2_38, 3_37, 0_41, 2_39, 3_38, 0_42, 2_40, 3_39, 0_43, 2_41, 3_40, 0_44, 2_42, 3_41, 0_45, 2_43, 3_42, 0_46, 2_44, 3_43, 0_47, 2_45, 3_44, 0_48, 2_46, 3_45, 0_49, 2_47, 3_46, 0_50, 2_48, 3_47, 0_51, 2_49, 3_48, 0_52, 2_50, 3_49, 0_53, 2_51, 3_50, 0_54, 2_52, 3_51, 0_55, 2_53, 3_52, 0_56, 2_54, 3_53, 0_57, 2_55, 3_54, 0_58, 2_56, 3_55, 0_59, 2_57, 3_56, 0_60, 2_58, 3_57, 0_61, 2_59, 3_58, 0_62, 2_60, 3_59, 0_63, 2_61, 3_60, 0_64, 2_62, 3_61, 0_65, 2_63, 3_62, 0_66, 2_64, 3_63, 0_67, 2_65, 3_64, 0_68, 2_66, 3_65, 0_69, 2_67, 3_66, 0_70, 2_68, 3_67, 0_71, 2_69, 3_68, 0_72, 2_70, 3_69, 0_73, 2_71, 3_70, 0_74, 2_72, 3_71, 0_75, 2_73, 3_72, 0_76, 2_74, 3_73, 0_77, 2_75, 3_74, 0_78, 2_76, 3_75, 0_79, 2_77, 3_76, 0_80, 2_78, 3_77, 0_81, 2_79, 3_78, 0_82, 2_80, 3_79, 0_83, 2_81, 3_80, 0_84, 2_82, 3_81, 0_85, 2_83, 3_82, 0_86, 2_84, 3_83, 0_87, 2_85, 3_84, 0_88, 2_86, 3_85, 0_89, 2_87, 3_86, 0_90, 2_88, 3_87, 0_91, 2_89, 3_88, 0_92, 2_90, 3_89, 0_93, 2_91, 3_90, 0_94, 2_92, 3_91, 0_95, 2_93, 3_92, 0_96, 2_94, 3_93, 0_97, 2_95, 3_94, 0_98, 2_96, 3_95, 0_99, 2_97, 3_96, 2_98, 3_97, 2_99, 3_98, 3_99]) standbyTasks: ([]) assignedTasks: ([0_0, 0_1, 1_0, 0_2, 1_1, 2_0, 0_3, 1_2, 2_1, 3_0, 0_4, 2_2, 3_1, 0_5, 2_3, 3_2, 0_6, 2_4, 3_3, 0_7, 2_5, 3_4, 0_8, 2_6, 3_5, 0_9, 2_7, 3_6, 0_10, 2_8, 3_7, 0_11, 2_9, 3_8, 0_12, 2_10, 3_9, 0_13, 2_11, 3_10, 0_14, 2_12, 3_11, 0_15, 2_13, 3_12, 0_16, 2_14, 3_13, 0_17, 2_15, 3_14, 0_18, 2_16, 3_15, 0_19, 2_17, 3_16, 0_20, 2_18, 3_17, 0_21, 2_19, 3_18, 0_22, 2_20, 3_19, 0_23, 2_21, 3_20, 0_24, 2_22, 3_21, 0_25, 2_23, 3_22, 0_26, 2_24, 3_23, 0_27, 2_25, 3_24, 0_28, 2_26, 3_25, 0_29, 2_27, 3_26, 0_30, 2_28, 3_27, 0_31, 2_29, 3_28, 0_32, 2_30, 3_29, 0_33, 2_31, 3_30, 0_34, 2_32, 3_31, 0_35, 2_33, 3_32, 0_36, 2_34, 3_33, 0_37, 2_35, 3_34, 0_38, 2_36, 3_35, 0_39, 2_37, 3_36, 0_40, 2_38, 3_37, 0_41, 2_39, 3_38, 0_42, 2_40, 3_39, 0_43, 2_41, 3_40, 0_44, 2_42, 3_41, 0_45, 2_43, 3_42, 0_46, 2_44, 3_43, 0_47, 2_45, 3_44, 0_48, 2_46, 3_45, 0_49, 2_47, 3_46, 0_50, 2_48, 3_47, 0_51, 2_49, 3_48, 0_52, 2_50, 3_49, 0_53, 2_51, 3_50, 0_54, 2_52, 3_51, 0_55, 2_53, 3_52, 0_56, 2_54, 3_53, 0_57, 2_55, 3_54, 0_58, 2_56, 3_55, 0_59, 2_57, 3_56, 0_60, 2_58, 3_57, 0_61, 2_59, 3_58, 0_62, 2_60, 3_59, 0_63, 2_61, 3_60, 0_64, 2_62, 3_61, 0_65, 2_63, 3_62, 0_66, 2_64, 3_63, 0_67, 2_65, 3_64, 0_68, 2_66, 3_65, 0_69, 2_67, 3_66, 0_70, 2_68, 3_67, 0_71, 2_69, 3_68, 0_72, 2_70, 3_69, 0_73, 2_71, 3_70, 0_74, 2_72, 3_71, 0_75, 2_73, 3_72, 0_76, 2_74, 3_73, 0_77, 2_75, 3_74, 0_78, 2_76, 3_75, 0_79, 2_77, 3_76, 0_80, 2_78, 3_77, 0_81, 2_79, 3_78, 0_82, 2_80, 3_79, 0_83, 2_81, 3_80, 0_84, 2_82, 3_81, 0_85, 2_83, 3_82, 0_86, 2_84, 3_83, 0_87, 2_85, 3_84, 0_88, 2_86, 3_85, 0_89, 2_87, 3_86, 0_90, 2_88, 3_87, 0_91, 2_89, 3_88, 0_92, 2_90, 3_89, 0_93, 2_91, 3_90, 0_94, 2_92, 3_91, 0_95, 2_93, 3_92, 0_96, 2_94, 3_93, 0_97, 2_95, 3_94, 0_98, 2_96, 3_95, 0_99, 2_97, 3_96, 2_98, 3_97, 2_99, 3_98, 3_99]) prevActiveTasks: ([]) prevStandbyTasks: ([0_0, 1_0, 0_1, 1_1, 0_2, 2_0, 1_2, 0_3, 2_1, 3_0, 0_4, 2_2, 3_1, 0_5, 2_3, 3_2, 0_6, 2_4, 3_3, 0_7, 2_5, 3_4, 0_8, 2_6, 3_5, 0_9, 2_7, 3_6, 0_10, 2_8, 3_7, 0_11, 2_9, 3_8, 0_12, 2_10, 3_9, 0_13, 2_11, 3_10, 0_14, 2_12, 3_11, 0_15, 2_13, 3_12, 0_16, 2_14, 3_13, 0_17, 2_15, 3_14, 0_18, 2_16, 3_15, 0_19, 2_17, 3_16, 0_20, 2_18, 3_17, 0_21, 2_19, 3_18, 0_22, 2_20, 3_19, 0_23, 2_21, 3_20, 0_24, 2_22, 3_21, 0_25, 2_23, 3_22, 0_26, 2_24, 3_23, 0_27, 2_25, 3_24, 0_28, 2_26, 3_25, 0_29, 2_27, 3_26, 0_30, 2_28, 3_27, 0_31, 2_29, 3_28, 0_32, 2_30, 3_29, 0_33, 2_31, 3_30, 0_34, 2_32, 3_31, 0_35, 2_33, 3_32, 0_36, 2_34, 3_33, 0_37, 2_35, 3_34, 0_38, 2_36, 3_35, 0_39, 2_37, 3_36, 0_40, 2_38, 3_37, 0_41, 2_39, 3_38, 0_42, 2_40, 3_39, 0_43, 2_41, 3_40, 0_44, 2_42, 3_41, 0_45, 2_43, 3_42, 0_46, 2_44, 3_43, 0_47, 2_45, 3_44, 0_48, 2_46, 3_45, 0_49, 2_47, 3_46, 0_50, 2_48, 3_47, 0_51, 2_49, 3_48, 0_52, 2_50, 3_49, 0_53, 2_51, 3_50, 0_54, 2_52, 3_51, 0_55, 2_53, 3_52, 0_56, 2_54, 3_53, 0_57, 2_55, 3_54, 0_58, 2_56, 3_55, 0_59, 2_57, 3_56, 0_60, 2_58, 3_57, 0_61, 2_59, 3_58, 0_62, 2_60, 3_59, 0_63, 2_61, 3_60, 0_64, 2_62, 3_61, 0_65, 2_63, 3_62, 0_66, 2_64, 3_63, 0_67, 2_65, 3_64, 0_68, 2_66, 3_65, 0_69, 2_67, 3_66, 0_70, 2_68, 3_67, 0_71, 2_69, 3_68, 0_72, 2_70, 3_69, 0_73, 2_71, 3_70, 0_74, 2_72, 3_71, 0_75, 2_73, 3_72, 0_76, 2_74, 3_73, 0_77, 2_75, 3_74, 0_78, 2_76, 3_75, 0_79, 2_77, 3_76, 0_80, 2_78, 3_77, 0_81, 2_79, 3_78, 0_82, 2_80, 3_79, 0_83, 2_81, 3_80, 0_84, 2_82, 3_81, 0_85, 2_83, 3_82, 0_86, 2_84, 3_83, 0_87, 2_85, 3_84, 0_88, 2_86, 3_85, 0_89, 2_87, 3_86, 0_90, 2_88, 3_87, 0_91, 2_89, 3_88, 0_92, 2_90, 3_89, 0_93, 2_91, 3_90, 0_94, 2_92, 3_91, 0_95, 2_93, 3_92, 0_96, 2_94, 3_93, 0_97, 2_95, 3_94, 0_98, 2_96, 3_95, 0_99, 2_97, 3_96, 2_98, 3_97, 2_99, 3_98, 3_99]) prevAssignedTasks: ([0_0, 1_0, 0_1, 1_1, 0_2, 2_0, 1_2, 0_3, 2_1, 3_0, 0_4, 2_2, 3_1, 0_5, 2_3, 3_2, 0_6, 2_4, 3_3, 0_7, 2_5, 3_4, 0_8, 2_6, 3_5, 0_9, 2_7, 3_6, 0_10, 2_8, 3_7, 0_11, 2_9, 3_8, 0_12, 2_10, 3_9, 0_13, 2_11, 3_10, 0_14, 2_12, 3_11, 0_15, 2_13, 3_12, 0_16, 2_14, 3_13, 0_17, 2_15, 3_14, 0_18, 2_16, 3_15, 0_19, 2_17, 3_16, 0_20, 2_18, 3_17, 0_21, 2_19, 3_18, 0_22, 2_20, 3_19, 0_23, 2_21, 3_20, 0_24, 2_22, 3_21, 0_25, 2_23, 3_22, 0_26, 2_24, 3_23, 0_27, 2_25, 3_24, 0_28, 2_26, 3_25, 0_29, 2_27, 3_26, 0_30, 2_28, 3_27, 0_31, 2_29, 3_28, 0_32, 2_30, 3_29, 0_33, 2_31, 3_30, 0_34, 2_32, 3_31, 0_35, 2_33, 3_32, 0_36, 2_34, 3_33, 0_37, 2_35, 3_34, 0_38, 2_36, 3_35, 0_39, 2_37, 3_36, 0_40, 2_38, 3_37, 0_41, 2_39, 3_38, 0_42, 2_40, 3_39, 0_43, 2_41, 3_40, 0_44, 2_42, 3_41, 0_45, 2_43, 3_42, 0_46, 2_44, 3_43, 0_47, 2_45, 3_44, 0_48, 2_46, 3_45, 0_49, 2_47, 3_46, 0_50, 2_48, 3_47, 0_51, 2_49, 3_48, 0_52, 2_50, 3_49, 0_53, 2_51, 3_50, 0_54, 2_52, 3_51, 0_55, 2_53, 3_52, 0_56, 2_54, 3_53, 0_57, 2_55, 3_54, 0_58, 2_56, 3_55, 0_59, 2_57, 3_56, 0_60, 2_58, 3_57, 0_61, 2_59, 3_58, 0_62, 2_60, 3_59, 0_63, 2_61, 3_60, 0_64, 2_62, 3_61, 0_65, 2_63, 3_62, 0_66, 2_64, 3_63, 0_67, 2_65, 3_64, 0_68, 2_66, 3_65, 0_69, 2_67, 3_66, 0_70, 2_68, 3_67, 0_71, 2_69, 3_68, 0_72, 2_70, 3_69, 0_73, 2_71, 3_70, 0_74, 2_72, 3_71, 0_75, 2_73, 3_72, 0_76, 2_74, 3_73, 0_77, 2_75, 3_74, 0_78, 2_76, 3_75, 0_79, 2_77, 3_76, 0_80, 2_78, 3_77, 0_81, 2_79, 3_78, 0_82, 2_80, 3_79, 0_83, 2_81, 3_80, 0_84, 2_82, 3_81, 0_85, 2_83, 3_82, 0_86, 2_84, 3_83, 0_87, 2_85, 3_84, 0_88, 2_86, 3_85, 0_89, 2_87, 3_86, 0_90, 2_88, 3_87, 0_91, 2_89, 3_88, 0_92, 2_90, 3_89, 0_93, 2_91, 3_90, 0_94, 2_92, 3_91, 0_95, 2_93, 3_92, 0_96, 2_94, 3_93, 0_97, 2_95, 3_94, 0_98, 2_96, 3_95, 0_99, 2_97, 3_96, 2_98, 3_97, 2_99, 3_98, 3_99]) capacity: 1]}.
> 2019-10-01 13:05:36,777 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.AbstractCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Successfully joined group with generation 5
> 2019-10-01 13:05:36,924 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Setting newly assigned partitions: arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-59, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-92, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-26, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-22, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-89, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-15, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-48, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-23, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-81, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-56, arena-rg-file-events-v3-72, arena-rg-file-events-v3-6, arena-rg-file-events-v3-39, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-91, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-25, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-58, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-21, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-54, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-14, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-47, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-80, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-55, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-88, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-24, arena-rg-file-events-v3-5, arena-rg-file-events-v3-38, arena-rg-file-events-v3-71, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-57, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-90, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-91, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-13, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-46, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-25, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-79, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-58, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-23, arena-rg-file-events-v3-4, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-56, arena-rg-file-events-v3-37, arena-rg-file-events-v3-70, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-89, arena-rg-file-events-v3-3, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-12, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-45, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-78, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-57, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-90, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-24, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-55, arena-rg-file-events-v3-36, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-88, arena-rg-file-events-v3-69, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-22, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-51, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-84, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-18, arena-rg-file-events-v3-76, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-19, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-52, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-85, arena-rg-file-events-v3-10, arena-rg-file-events-v3-43, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-87, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-21, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-54, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-83, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-17, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-50, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-18, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-51, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-20, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-84, arena-rg-file-events-v3-9, arena-rg-file-events-v3-42, arena-rg-file-events-v3-75, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-53, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-86, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-53, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-86, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-20, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-87, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-17, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-50, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-19, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-83, arena-rg-file-events-v3-8, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-52, arena-rg-file-events-v3-41, arena-rg-file-events-v3-74, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-85, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-85, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-19, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-52, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-16, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-49, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-82, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-51, arena-rg-file-events-v3-40, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-84, arena-rg-file-events-v3-73, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-18, arena-rg-file-events-v3-7, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-67, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-1, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-34, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-7, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-40, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-73, arena-rg-file-events-v3-80, arena-rg-file-events-v3-14, arena-rg-file-events-v3-47, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-97, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-31, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-64, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-0, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-99, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-33, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-66, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-6, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-39, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-72, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-30, arena-rg-file-events-v3-13, arena-rg-file-events-v3-46, arena-rg-file-events-v3-79, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-63, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-96, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-32, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-65, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-5, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-98, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-38, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-71, arena-rg-file-events-v3-12, arena-rg-file-events-v3-45, arena-rg-file-events-v3-78, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-99, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-33, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-66, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-0, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-31, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-64, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-4, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-97, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-37, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-70, arena-rg-file-events-v3-44, arena-rg-file-events-v3-77, arena-rg-file-events-v3-11, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-65, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-98, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-32, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-63, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-96, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-30, arena-rg-file-events-v3-52, arena-rg-file-events-v3-85, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-93, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-11, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-44, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-27, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-77, arena-rg-file-events-v3-18, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-60, arena-rg-file-events-v3-51, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-95, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-29, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-62, arena-rg-file-events-v3-84, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-10, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-43, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-76, arena-rg-file-events-v3-17, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-59, arena-rg-file-events-v3-50, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-92, arena-rg-file-events-v3-83, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-26, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-28, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-61, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-94, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-9, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-42, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-29, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-75, arena-rg-file-events-v3-16, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-62, arena-rg-file-events-v3-49, arena-rg-file-events-v3-82, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-95, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-27, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-60, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-93, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-8, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-41, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-74, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-61, arena-rg-file-events-v3-48, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-94, arena-rg-file-events-v3-81, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-28, arena-rg-file-events-v3-15, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-32, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-39, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-65, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-72, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-98, arena-rg-file-events-v3-56, arena-rg-file-events-v3-89, arena-rg-file-events-v3-23, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-75, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-9, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-42, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-6, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-31, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-64, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-71, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-97, arena-rg-file-events-v3-88, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-8, arena-rg-file-events-v3-22, arena-rg-file-events-v3-55, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-41, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-74, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-5, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-38, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-30, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-41, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-63, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-74, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-96, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-8, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-7, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-40, arena-rg-file-events-v3-21, arena-rg-file-events-v3-54, arena-rg-file-events-v3-87, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-73, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-29, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-62, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-73, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-95, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-7, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-40, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-39, arena-rg-file-events-v3-20, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-72, arena-rg-file-events-v3-53, arena-rg-file-events-v3-86, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-6, arena-rg-file-events-v3-19, arena-rg-file-events-v3-60, arena-rg-file-events-v3-93, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-3, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-36, arena-rg-dlq-reporting-events-1, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-69, arena-rg-file-events-v3-27, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-71, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-5, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-38, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-35, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-68, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-2, arena-rg-file-events-v3-92, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-2, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-35, arena-rg-dlq-reporting-events-0, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-4, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-68, arena-rg-file-events-v3-26, arena-rg-file-events-v3-59, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-37, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-70, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-67, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-1, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-34, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-1, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-34, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-3, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-67, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-36, arena-rg-file-events-v3-25, arena-rg-file-events-v3-58, arena-rg-file-events-v3-91, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-69, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-37, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-70, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-4, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-0, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-33, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-66, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-35, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-99, arena-rg-file-events-v3-24, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-68, arena-rg-file-events-v3-57, arena-rg-file-events-v3-90, arena-rg-dlq-reporting-events-2, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-2, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-69, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-3, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-36, arena-rg-file-events-v3-64, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-14, arena-rg-file-events-v3-97, arena-rg-file-events-v3-31, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-47, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-80, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-83, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-17, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-50, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-23, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-56, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-89, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-13, arena-rg-file-events-v3-96, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-46, arena-rg-file-events-v3-30, arena-rg-file-events-v3-63, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-79, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-16, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-49, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-82, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-22, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-55, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-88, arena-rg-file-events-v3-29, arena-rg-file-events-v3-62, arena-rg-file-events-v3-95, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-49, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-82, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-16, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-15, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-48, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-81, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-21, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-54, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-87, arena-rg-file-events-v3-28, arena-rg-file-events-v3-61, arena-rg-file-events-v3-94, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-81, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-15, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-48, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-47, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-80, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-14, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-20, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-53, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-86, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-28, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-43, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-61, arena-rg-file-events-v3-2, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-76, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-94, arena-rg-file-events-v3-35, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-10, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-79, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-13, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-46, arena-rg-file-events-v3-68, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-27, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-60, arena-rg-file-events-v3-1, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-75, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-93, arena-rg-file-events-v3-34, arena-rg-file-events-v3-67, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-9, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-42, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-12, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-45, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-78, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-26, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-45, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-59, arena-rg-file-events-v3-0, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-78, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-92, arena-rg-file-events-v3-33, arena-rg-file-events-v3-66, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-12, arena-rg-file-events-v3-99, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-11, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-44, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-77, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-25, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-58, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-77, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-91, arena-rg-file-events-v3-32, arena-rg-file-events-v3-65, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-11, arena-rg-file-events-v3-98, arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-44, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-43, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-76, arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-10, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-24, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-57, arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-90
> 2019-10-01 13:05:36,925 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] State transition from PARTITIONS_REVOKED to PARTITIONS_ASSIGNED
> 2019-10-01 13:05:37,352 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.str.pro.int.StreamThread              : stream-thread [arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1] partition assignment took 427 ms.
> 	current active tasks: [0_0, 0_1, 1_0, 2_0, 0_2, 1_1, 3_0, 2_1, 0_3, 1_2, 3_1, 2_2, 0_4, 3_2, 2_3, 0_5, 3_3, 2_4, 0_6, 3_4, 2_5, 0_7, 3_5, 2_6, 0_8, 3_6, 2_7, 0_9, 3_7, 2_8, 0_10, 3_8, 2_9, 0_11, 3_9, 2_10, 0_12, 3_10, 2_11, 0_13, 3_11, 2_12, 0_14, 3_12, 2_13, 0_15, 3_13, 2_14, 0_16, 3_14, 2_15, 0_17, 3_15, 2_16, 0_18, 3_16, 2_17, 0_19, 3_17, 2_18, 0_20, 3_18, 2_19, 0_21, 3_19, 2_20, 0_22, 3_20, 2_21, 0_23, 3_21, 2_22, 0_24, 3_22, 2_23, 0_25, 3_23, 2_24, 0_26, 3_24, 2_25, 0_27, 3_25, 2_26, 0_28, 3_26, 2_27, 0_29, 3_27, 2_28, 0_30, 3_28, 2_29, 0_31, 3_29, 2_30, 0_32, 3_30, 2_31, 0_33, 3_31, 2_32, 0_34, 3_32, 2_33, 0_35, 3_33, 2_34, 0_36, 3_34, 2_35, 0_37, 3_35, 2_36, 0_38, 3_36, 2_37, 0_39, 3_37, 2_38, 0_40, 3_38, 2_39, 0_41, 3_39, 2_40, 0_42, 3_40, 2_41, 0_43, 3_41, 2_42, 0_44, 3_42, 2_43, 0_45, 3_43, 2_44, 0_46, 3_44, 2_45, 0_47, 3_45, 2_46, 0_48, 3_46, 2_47, 0_49, 3_47, 2_48, 0_50, 3_48, 2_49, 0_51, 3_49, 2_50, 0_52, 3_50, 2_51, 0_53, 3_51, 2_52, 0_54, 3_52, 2_53, 0_55, 3_53, 2_54, 0_56, 3_54, 2_55, 0_57, 3_55, 2_56, 0_58, 3_56, 2_57, 0_59, 3_57, 2_58, 0_60, 3_58, 2_59, 0_61, 3_59, 2_60, 0_62, 3_60, 2_61, 0_63, 3_61, 2_62, 0_64, 3_62, 2_63, 0_65, 3_63, 2_64, 0_66, 3_64, 2_65, 0_67, 3_65, 2_66, 0_68, 3_66, 2_67, 0_69, 3_67, 2_68, 0_70, 3_68, 2_69, 0_71, 3_69, 2_70, 0_72, 3_70, 2_71, 0_73, 3_71, 2_72, 0_74, 3_72, 2_73, 0_75, 3_73, 2_74, 0_76, 3_74, 2_75, 0_77, 3_75, 2_76, 0_78, 3_76, 2_77, 0_79, 3_77, 2_78, 0_80, 3_78, 2_79, 0_81, 3_79, 2_80, 0_82, 3_80, 2_81, 0_83, 3_81, 2_82, 0_84, 3_82, 2_83, 0_85, 3_83, 2_84, 0_86, 3_84, 2_85, 0_87, 3_85, 2_86, 0_88, 3_86, 2_87, 0_89, 3_87, 2_88, 0_90, 3_88, 2_89, 0_91, 3_89, 2_90, 0_92, 3_90, 2_91, 0_93, 3_91, 2_92, 0_94, 3_92, 2_93, 0_95, 3_93, 2_94, 0_96, 3_94, 2_95, 0_97, 3_95, 2_96, 0_98, 3_96, 2_97, 0_99, 3_97, 2_98, 3_98, 2_99, 3_99]
> 	current standby tasks: []
> 	previous active tasks: []
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-59
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-92
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-26
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-22
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-15
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-89
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-48
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-81
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-23
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-56
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-72
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-6
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-39
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-91
> 2019-10-01 13:05:37,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-25
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-58
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-21
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-54
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-14
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-47
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-80
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-55
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-88
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-5
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-24
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-38
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-71
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-57
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-90
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-13
> 2019-10-01 13:05:37,399 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-91
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-46
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-79
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-25
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-58
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-4
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-23
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-37
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-56
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-70
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-89
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-3
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-12
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-45
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-78
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-57
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-90
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-24
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-36
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-55
> 2019-10-01 13:05:37,400 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-69
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-88
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-22
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-51
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-84
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-18
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-76
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-19
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-52
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-10
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-85
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-43
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-87
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-21
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-54
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-83
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-17
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-50
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-18
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-51
> 2019-10-01 13:05:37,401 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-9
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-84
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-20
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-42
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-75
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-53
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-86
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-53
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-86
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-20
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-17
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-87
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-50
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-8
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-83
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-19
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-41
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-52
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-74
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-85
> 2019-10-01 13:05:37,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-85
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-19
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-52
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-16
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-49
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-82
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-40
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-51
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-73
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-84
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-18
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-7
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-67
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-1
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-34
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-7
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-40
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-73
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-80
> 2019-10-01 13:05:37,403 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-14
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-47
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-97
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-31
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-64
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-0
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-99
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-33
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-6
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-66
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-39
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-72
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-30
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-13
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-46
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-79
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-63
> 2019-10-01 13:05:37,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-96
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-32
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-5
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-65
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-38
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-98
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-71
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-12
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-45
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-78
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-99
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-33
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-66
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-0
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-31
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-64
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-4
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-37
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-97
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-70
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-44
> 2019-10-01 13:05:37,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-77
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-11
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-65
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-98
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-32
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-63
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-96
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-30
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-52
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-85
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-11
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-93
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-44
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-18
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-77
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-27
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-51
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-60
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-95
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-29
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-62
> 2019-10-01 13:05:37,406 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-84
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-10
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-43
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-17
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-76
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-50
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-59
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-83
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-92
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-26
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-28
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-61
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-94
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-9
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-42
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-16
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-75
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-29
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-49
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-62
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-82
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-95
> 2019-10-01 13:05:37,407 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-27
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-60
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-93
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-8
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-41
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-74
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-48
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-61
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-81
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-94
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-28
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-15
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-32
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-65
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-39
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-98
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-72
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-56
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-89
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-23
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-75
> 2019-10-01 13:05:37,408 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-9
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-42
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-6
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-31
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-64
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-97
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-71
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-88
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-8
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-22
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-55
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-41
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-74
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-5
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-38
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-30
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-63
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-41
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-96
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-74
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-8
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-7
> 2019-10-01 13:05:37,409 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-21
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-40
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-54
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-87
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-73
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-29
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-62
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-95
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-73
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-7
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-40
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-20
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-39
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-53
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-72
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-86
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-6
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-19
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-60
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-93
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-3
> 2019-10-01 13:05:37,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-36
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-dlq-reporting-events-1
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-69
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-27
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-71
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-5
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-38
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-35
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-68
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-2
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-92
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-2
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-35
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-dlq-reporting-events-0
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-68
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-4
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-26
> 2019-10-01 13:05:37,411 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-59
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-37
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-70
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-67
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-1
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-34
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-1
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-34
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-67
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-3
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-25
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-36
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-58
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-91
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-69
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-37
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-70
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-4
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-0
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-33
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-66
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-24
> 2019-10-01 13:05:37,412 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-99
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-35
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-57
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-68
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-90
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-dlq-reporting-events-2
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-2
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-69
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-3
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-36
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-64
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-97
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-14
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-31
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-47
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-80
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-83
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-17
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-50
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-23
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-56
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-89
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-96
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-13
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-46
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-30
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-63
> 2019-10-01 13:05:37,413 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-79
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-16
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-49
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-22
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-82
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-55
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-88
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-29
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-62
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-95
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-49
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-82
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-16
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-15
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-48
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-21
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-81
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-54
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-87
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-28
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-61
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-94
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-81
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-15
> 2019-10-01 13:05:37,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-48
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-47
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-80
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-14
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-20
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-53
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-86
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-28
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-2
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-61
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-43
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-35
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-94
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-76
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-10
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-79
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-13
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-46
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-68
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-27
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-1
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-60
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-34
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-93
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-75
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-67
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-9
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-42
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-12
> 2019-10-01 13:05:37,415 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-45
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-78
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-26
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-0
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-59
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-45
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-33
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-92
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-78
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-66
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-99
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-12
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-11
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-44
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-77
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-25
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-58
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-32
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-91
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-77
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-65
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-98
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-11
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-44
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-43
> 2019-10-01 13:05:37,416 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-76
> 2019-10-01 13:05:37,417 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-10
> 2019-10-01 13:05:37,417 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-24
> 2019-10-01 13:05:37,417 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-57
> 2019-10-01 13:05:37,417 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-90
> 2019-10-01 13:05:37,417 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-0
> 2019-10-01 13:05:37,466 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-0
> 2019-10-01 13:05:37,499 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-0
> 2019-10-01 13:05:37,598 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-1
> 2019-10-01 13:05:37,623 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-1
> 2019-10-01 13:05:37,658 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-1
> 2019-10-01 13:05:37,688 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-2
> 2019-10-01 13:05:37,715 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-2
> 2019-10-01 13:05:37,752 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-2
> 2019-10-01 13:05:37,786 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-3
> 2019-10-01 13:05:37,813 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-3
> 2019-10-01 13:05:37,838 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-3
> 2019-10-01 13:05:37,867 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-4
> 2019-10-01 13:05:37,891 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-4
> 2019-10-01 13:05:37,918 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-4
> 2019-10-01 13:05:37,953 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-5
> 2019-10-01 13:05:37,976 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-5
> 2019-10-01 13:05:38,003 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-5
> 2019-10-01 13:05:38,030 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-6
> 2019-10-01 13:05:38,052 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-6
> 2019-10-01 13:05:38,077 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-6
> 2019-10-01 13:05:38,108 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-7
> 2019-10-01 13:05:38,167 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-7
> 2019-10-01 13:05:38,193 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-7
> 2019-10-01 13:05:38,268 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-8
> 2019-10-01 13:05:38,290 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-8
> 2019-10-01 13:05:38,366 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-8
> 2019-10-01 13:05:38,395 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-9
> 2019-10-01 13:05:38,466 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-9
> 2019-10-01 13:05:38,499 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-9
> 2019-10-01 13:05:38,565 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-10
> 2019-10-01 13:05:38,587 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-10
> 2019-10-01 13:05:38,666 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-10
> 2019-10-01 13:05:38,696 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-11
> 2019-10-01 13:05:38,719 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-11
> 2019-10-01 13:05:38,744 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-11
> 2019-10-01 13:05:38,774 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-12
> 2019-10-01 13:05:38,796 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-12
> 2019-10-01 13:05:38,821 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-12
> 2019-10-01 13:05:38,854 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-13
> 2019-10-01 13:05:38,875 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-13
> 2019-10-01 13:05:38,899 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-13
> 2019-10-01 13:05:38,930 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-14
> 2019-10-01 13:05:38,952 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-14
> 2019-10-01 13:05:38,976 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-14
> 2019-10-01 13:05:39,005 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-15
> 2019-10-01 13:05:39,029 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-15
> 2019-10-01 13:05:39,070 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-15
> 2019-10-01 13:05:39,100 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-16
> 2019-10-01 13:05:39,122 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-16
> 2019-10-01 13:05:39,152 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-16
> 2019-10-01 13:05:39,180 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-17
> 2019-10-01 13:05:39,202 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-17
> 2019-10-01 13:05:39,228 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-17
> 2019-10-01 13:05:39,261 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-18
> 2019-10-01 13:05:39,285 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-18
> 2019-10-01 13:05:39,315 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-18
> 2019-10-01 13:05:39,347 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-19
> 2019-10-01 13:05:39,372 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-19
> 2019-10-01 13:05:39,398 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-19
> 2019-10-01 13:05:39,428 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-20
> 2019-10-01 13:05:39,452 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-20
> 2019-10-01 13:05:39,487 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-20
> 2019-10-01 13:05:39,515 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-21
> 2019-10-01 13:05:39,538 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-21
> 2019-10-01 13:05:39,568 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-21
> 2019-10-01 13:05:39,598 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-22
> 2019-10-01 13:05:39,627 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-22
> 2019-10-01 13:05:39,653 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-22
> 2019-10-01 13:05:39,683 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-23
> 2019-10-01 13:05:39,707 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-23
> 2019-10-01 13:05:39,733 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-23
> 2019-10-01 13:05:39,761 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-24
> 2019-10-01 13:05:39,787 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-24
> 2019-10-01 13:05:39,817 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-24
> 2019-10-01 13:05:39,847 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-25
> 2019-10-01 13:05:39,869 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-25
> 2019-10-01 13:05:39,896 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-25
> 2019-10-01 13:05:39,925 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-26
> 2019-10-01 13:05:39,948 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-26
> 2019-10-01 13:05:39,974 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-26
> 2019-10-01 13:05:40,002 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-27
> 2019-10-01 13:05:40,032 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-27
> 2019-10-01 13:05:40,056 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-27
> 2019-10-01 13:05:40,092 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-28
> 2019-10-01 13:05:40,117 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-28
> 2019-10-01 13:05:40,142 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-28
> 2019-10-01 13:05:40,171 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-29
> 2019-10-01 13:05:40,193 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-29
> 2019-10-01 13:05:40,218 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-29
> 2019-10-01 13:05:40,246 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-30
> 2019-10-01 13:05:40,268 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-30
> 2019-10-01 13:05:40,293 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-30
> 2019-10-01 13:05:40,323 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-31
> 2019-10-01 13:05:40,347 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-31
> 2019-10-01 13:05:40,373 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-31
> 2019-10-01 13:05:40,402 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-32
> 2019-10-01 13:05:40,428 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-32
> 2019-10-01 13:05:40,456 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-32
> 2019-10-01 13:05:40,485 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-33
> 2019-10-01 13:05:40,508 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-33
> 2019-10-01 13:05:40,534 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-33
> 2019-10-01 13:05:40,562 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-34
> 2019-10-01 13:05:40,584 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-34
> 2019-10-01 13:05:40,610 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-34
> 2019-10-01 13:05:40,637 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-35
> 2019-10-01 13:05:40,661 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-35
> 2019-10-01 13:05:40,686 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-35
> 2019-10-01 13:05:40,716 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-36
> 2019-10-01 13:05:40,739 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-36
> 2019-10-01 13:05:40,764 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-36
> 2019-10-01 13:05:40,792 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-37
> 2019-10-01 13:05:40,814 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-37
> 2019-10-01 13:05:40,839 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-37
> 2019-10-01 13:05:40,877 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-38
> 2019-10-01 13:05:40,900 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-38
> 2019-10-01 13:05:40,930 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-38
> 2019-10-01 13:05:40,959 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-39
> 2019-10-01 13:05:40,980 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-39
> 2019-10-01 13:05:41,008 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-39
> 2019-10-01 13:05:41,037 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-40
> 2019-10-01 13:05:41,063 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-40
> 2019-10-01 13:05:41,093 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-40
> 2019-10-01 13:05:41,122 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-41
> 2019-10-01 13:05:41,144 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-41
> 2019-10-01 13:05:41,170 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-41
> 2019-10-01 13:05:41,198 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-42
> 2019-10-01 13:05:41,222 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-42
> 2019-10-01 13:05:41,247 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-42
> 2019-10-01 13:05:41,279 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-43
> 2019-10-01 13:05:41,301 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-43
> 2019-10-01 13:05:41,334 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-43
> 2019-10-01 13:05:41,362 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-44
> 2019-10-01 13:05:41,384 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-44
> 2019-10-01 13:05:41,410 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-44
> 2019-10-01 13:05:41,438 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-45
> 2019-10-01 13:05:41,465 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-45
> 2019-10-01 13:05:41,493 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-45
> 2019-10-01 13:05:41,522 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-46
> 2019-10-01 13:05:41,548 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-46
> 2019-10-01 13:05:41,571 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-46
> 2019-10-01 13:05:41,599 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-47
> 2019-10-01 13:05:41,622 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-47
> 2019-10-01 13:05:41,646 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-47
> 2019-10-01 13:05:41,679 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-48
> 2019-10-01 13:05:41,702 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-48
> 2019-10-01 13:05:41,726 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-48
> 2019-10-01 13:05:41,851 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-49
> 2019-10-01 13:05:41,875 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-49
> 2019-10-01 13:05:41,900 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-49
> 2019-10-01 13:05:41,928 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-50
> 2019-10-01 13:05:41,949 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-50
> 2019-10-01 13:05:41,974 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-50
> 2019-10-01 13:05:42,004 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-51
> 2019-10-01 13:05:42,028 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-51
> 2019-10-01 13:05:42,053 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-51
> 2019-10-01 13:05:42,081 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-52
> 2019-10-01 13:05:42,105 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-52
> 2019-10-01 13:05:42,135 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-52
> 2019-10-01 13:05:42,164 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-53
> 2019-10-01 13:05:42,187 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-53
> 2019-10-01 13:05:42,213 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-53
> 2019-10-01 13:05:42,266 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-54
> 2019-10-01 13:05:42,288 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-54
> 2019-10-01 13:05:42,314 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-54
> 2019-10-01 13:05:42,346 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-55
> 2019-10-01 13:05:42,368 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-55
> 2019-10-01 13:05:42,393 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-55
> 2019-10-01 13:05:42,424 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-56
> 2019-10-01 13:05:42,447 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-56
> 2019-10-01 13:05:42,474 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-56
> 2019-10-01 13:05:42,505 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-57
> 2019-10-01 13:05:42,527 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-57
> 2019-10-01 13:05:42,553 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-57
> 2019-10-01 13:05:42,581 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-58
> 2019-10-01 13:05:42,608 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-58
> 2019-10-01 13:05:42,639 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-58
> 2019-10-01 13:05:42,667 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-59
> 2019-10-01 13:05:42,690 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-59
> 2019-10-01 13:05:42,716 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-59
> 2019-10-01 13:05:42,747 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-60
> 2019-10-01 13:05:42,770 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-60
> 2019-10-01 13:05:42,794 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-60
> 2019-10-01 13:05:42,825 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-61
> 2019-10-01 13:05:42,851 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-61
> 2019-10-01 13:05:42,876 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-61
> 2019-10-01 13:05:42,903 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-62
> 2019-10-01 13:05:42,927 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-62
> 2019-10-01 13:05:42,959 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-62
> 2019-10-01 13:05:42,987 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-63
> 2019-10-01 13:05:43,010 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-63
> 2019-10-01 13:05:43,037 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-63
> 2019-10-01 13:05:43,066 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-64
> 2019-10-01 13:05:43,088 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-64
> 2019-10-01 13:05:43,120 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-64
> 2019-10-01 13:05:43,149 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-65
> 2019-10-01 13:05:43,177 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-65
> 2019-10-01 13:05:43,202 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-65
> 2019-10-01 13:05:43,277 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-66
> 2019-10-01 13:05:43,303 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-66
> 2019-10-01 13:05:43,331 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-66
> 2019-10-01 13:05:43,359 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-67
> 2019-10-01 13:05:43,381 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-67
> 2019-10-01 13:05:43,405 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-67
> 2019-10-01 13:05:43,436 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-68
> 2019-10-01 13:05:43,459 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-68
> 2019-10-01 13:05:43,488 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-68
> 2019-10-01 13:05:43,516 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-69
> 2019-10-01 13:05:43,543 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-69
> 2019-10-01 13:05:43,568 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-69
> 2019-10-01 13:05:43,596 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-70
> 2019-10-01 13:05:43,618 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-70
> 2019-10-01 13:05:43,647 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-70
> 2019-10-01 13:05:43,687 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-71
> 2019-10-01 13:05:43,716 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-71
> 2019-10-01 13:05:43,740 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-71
> 2019-10-01 13:05:43,769 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-72
> 2019-10-01 13:05:43,796 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-72
> 2019-10-01 13:05:43,824 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-72
> 2019-10-01 13:05:43,858 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-73
> 2019-10-01 13:05:43,880 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-73
> 2019-10-01 13:05:43,904 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-73
> 2019-10-01 13:05:43,933 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-74
> 2019-10-01 13:05:43,973 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-74
> 2019-10-01 13:05:43,999 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-74
> 2019-10-01 13:05:44,028 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-75
> 2019-10-01 13:05:44,050 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-75
> 2019-10-01 13:05:44,075 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-75
> 2019-10-01 13:05:44,106 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-76
> 2019-10-01 13:05:44,130 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-76
> 2019-10-01 13:05:44,154 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-76
> 2019-10-01 13:05:44,182 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-77
> 2019-10-01 13:05:44,205 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-77
> 2019-10-01 13:05:44,235 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-77
> 2019-10-01 13:05:44,264 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-78
> 2019-10-01 13:05:44,295 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-78
> 2019-10-01 13:05:44,323 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-78
> 2019-10-01 13:05:44,350 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-79
> 2019-10-01 13:05:44,373 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-79
> 2019-10-01 13:05:44,404 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-79
> 2019-10-01 13:05:44,435 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-80
> 2019-10-01 13:05:44,457 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-80
> 2019-10-01 13:05:44,482 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-80
> 2019-10-01 13:05:44,509 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-81
> 2019-10-01 13:05:44,532 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-81
> 2019-10-01 13:05:44,558 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-81
> 2019-10-01 13:05:44,587 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-82
> 2019-10-01 13:05:44,614 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-82
> 2019-10-01 13:05:44,642 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-82
> 2019-10-01 13:05:44,670 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-83
> 2019-10-01 13:05:44,692 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-83
> 2019-10-01 13:05:44,717 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-83
> 2019-10-01 13:05:44,746 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-84
> 2019-10-01 13:05:44,769 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-84
> 2019-10-01 13:05:44,794 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-84
> 2019-10-01 13:05:44,825 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-85
> 2019-10-01 13:05:44,847 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-85
> 2019-10-01 13:05:44,872 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-85
> 2019-10-01 13:05:44,900 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-86
> 2019-10-01 13:05:44,922 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-86
> 2019-10-01 13:05:44,947 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-86
> 2019-10-01 13:05:44,977 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-87
> 2019-10-01 13:05:45,000 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-87
> 2019-10-01 13:05:45,024 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-87
> 2019-10-01 13:05:45,056 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-88
> 2019-10-01 13:05:45,080 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-88
> 2019-10-01 13:05:45,110 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-88
> 2019-10-01 13:05:45,139 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-89
> 2019-10-01 13:05:45,162 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-89
> 2019-10-01 13:05:45,186 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-89
> 2019-10-01 13:05:45,216 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-90
> 2019-10-01 13:05:45,239 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-90
> 2019-10-01 13:05:45,263 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-90
> 2019-10-01 13:05:45,293 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-91
> 2019-10-01 13:05:45,313 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-91
> 2019-10-01 13:05:45,341 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-91
> 2019-10-01 13:05:45,367 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-92
> 2019-10-01 13:05:45,389 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-92
> 2019-10-01 13:05:45,414 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-92
> 2019-10-01 13:05:45,446 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-93
> 2019-10-01 13:05:45,470 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-93
> 2019-10-01 13:05:45,495 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-93
> 2019-10-01 13:05:45,524 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-94
> 2019-10-01 13:05:45,546 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-94
> 2019-10-01 13:05:45,574 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-94
> 2019-10-01 13:05:45,603 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-95
> 2019-10-01 13:05:45,626 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-95
> 2019-10-01 13:05:45,650 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-95
> 2019-10-01 13:05:45,680 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-96
> 2019-10-01 13:05:45,706 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-96
> 2019-10-01 13:05:45,730 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-96
> 2019-10-01 13:05:45,760 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-97
> 2019-10-01 13:05:45,784 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-97
> 2019-10-01 13:05:45,808 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-97
> 2019-10-01 13:05:45,836 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-98
> 2019-10-01 13:05:45,860 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-98
> 2019-10-01 13:05:45,892 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-98
> 2019-10-01 13:05:45,920 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-99
> 2019-10-01 13:05:45,944 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-99
> 2019-10-01 13:05:45,970 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-99
> 2019-10-01 13:05:46,087 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.Metadata                          : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-restore-consumer, groupId=null] Cluster ID: ZZ2bYOBRRDCV-nCSBUzyVQ
> 2019-10-01 13:06:16,287 INFO  --- [d9dd6-StreamThread-1] org.apa.kaf.cli.con.KafkaConsumer                 : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-restore-consumer, groupId=null] Unsubscribed all topics or patterns and assigned partitions
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-59
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-92
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-26
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-22
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-15
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-89
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-48
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-81
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-23
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-56
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-72
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-6
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-39
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-91
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-25
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-58
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-21
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-54
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-14
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-47
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-80
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-55
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-88
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-5
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-24
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-38
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-71
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-57
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-90
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-13
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-91
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-46
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-79
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-25
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-58
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-4
> 2019-10-01 13:06:16,333 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-23
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-37
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-56
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-70
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-89
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-3
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-12
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-45
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-78
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-57
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-90
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-24
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-36
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-55
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-69
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-88
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-22
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-51
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-84
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-18
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-76
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-19
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-52
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-10
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-85
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-43
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-87
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-21
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-54
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-83
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-17
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-50
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-18
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-51
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-9
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-84
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-20
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-42
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-75
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-53
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-86
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-53
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-86
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-20
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-17
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-87
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-50
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-8
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-83
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-19
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-41
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-52
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-74
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-85
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-85
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-19
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-52
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-16
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-49
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-82
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-40
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-51
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-73
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-84
> 2019-10-01 13:06:16,334 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-18
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-7
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-67
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-1
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-34
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-7
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-40
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-73
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-80
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-14
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-47
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-97
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-31
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-64
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-0
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-99
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-33
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-6
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-66
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-39
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-72
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-30
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-13
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-46
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-79
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-63
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-96
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-32
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-5
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-65
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-38
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-98
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-71
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-12
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-45
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-78
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-99
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-33
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-66
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-0
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-31
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-64
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-4
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-37
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-97
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-70
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-44
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-77
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-11
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-65
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-98
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-32
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-63
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-96
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-30
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-52
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-85
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-11
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-93
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-44
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-18
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-77
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-27
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-51
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-60
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-95
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-29
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-62
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-84
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-10
> 2019-10-01 13:06:16,335 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-43
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-17
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-76
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-50
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-59
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-83
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-92
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-26
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-28
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-61
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-94
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-9
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-42
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-16
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-75
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-29
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-49
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-62
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-82
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-95
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-27
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-60
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-93
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-8
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-41
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-74
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-48
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-61
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-81
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-94
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-28
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-15
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-32
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-65
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-39
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-98
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-72
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-56
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-89
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-23
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-75
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-9
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-42
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-6
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-31
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-64
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-97
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-71
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-88
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-8
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-22
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-55
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-41
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-74
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-5
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-38
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-30
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-63
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-41
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-96
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-74
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-8
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-7
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-21
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-40
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-54
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-87
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-73
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-29
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-62
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-95
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-73
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-7
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-40
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-20
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-39
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-53
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-72
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-86
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-6
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-19
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-60
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-93
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-3
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-36
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-dlq-reporting-events-1
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-69
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-27
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-71
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-5
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-38
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-35
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-68
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-2
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-92
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-2
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-35
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-dlq-reporting-events-0
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-68
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-4
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-26
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-59
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-37
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-70
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-67
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-1
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-34
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-1
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-34
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-67
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-3
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-25
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-36
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-58
> 2019-10-01 13:06:16,336 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-91
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-69
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-37
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-70
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-4
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-0
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-33
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-66
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-24
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-99
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-35
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-57
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-68
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-90
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-dlq-reporting-events-2
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-2
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-69
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-3
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-36
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-64
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-97
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-14
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-31
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-47
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-80
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-83
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-17
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-50
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-23
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-56
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-89
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-96
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-13
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-46
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-30
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-63
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-79
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-16
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-49
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-22
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-82
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-55
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-88
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-29
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-62
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-95
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-49
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-82
> 2019-10-01 13:06:16,337 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-16
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-15
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-48
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-21
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-81
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-54
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-87
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-28
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-61
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-94
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-81
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-15
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-48
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-47
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-80
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-14
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-20
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-53
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-86
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-28
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-2
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-61
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-43
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-35
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-94
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-76
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-10
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-79
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-13
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-46
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-68
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-27
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-1
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-60
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-34
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-93
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-75
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-67
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-9
> 2019-10-01 13:06:16,338 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-42
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-12
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-45
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-78
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-26
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-0
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-59
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-45
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-33
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-92
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-78
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-66
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-99
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-12
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-11
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-44
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-77
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-25
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-58
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-32
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-91
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-77
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-65
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-file-events-v3-98
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-11
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-MAP-0000000003-repartition-44
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-43
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-76
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-KSTREAM-KEY-SELECT-0000000001-repartition-10
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-24
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-57
> 2019-10-01 13:06:16,339 INFO  --- [leStatusStore-stream] org.apa.kaf.cli.con.int.ConsumerCoordinator       : [Consumer clientId=arena-rg-uiService-fileStatusStore-stream-ca9602d3-a249-4f1e-af1d-5c80246d9dd6-StreamThread-1-consumer, groupId=arena-rg-uiService-fileStatusStore-stream] Found no committed offset for partition arena-rg-uiService-fileStatusStore-stream-arena-rg-uiService-file-status-table-store-repartition-90
> It seems like a reset of the stream fixes the problem but why is this needed?



--
This message was sent by Atlassian Jira
(v8.3.4#803005)