Pranciskus Gecas created KAFKA-13232:
----------------------------------------

             Summary: UnknownTopicOrPartitionException for loading topic during 
broker startup.
                 Key: KAFKA-13232
                 URL: https://issues.apache.org/jira/browse/KAFKA-13232
             Project: Kafka
          Issue Type: Bug
          Components: core
    Affects Versions: 2.8.0, 2.7.0
            Reporter: Pranciskus Gecas


There is race condition in broker where topic that is loading returns 
org.apache.kafka.common.errors.UnknownTopicOrPartitionException.

Issue can be reproduced by producing to topic while broker is restarted. Broker 
log clearly shows that topic is being loaded, when it is reported as Unknown.

Issue reproduces only after running test multiple times.

{panel:title=Relevant log}
[2021-08-26 10:21:06,278] DEBUG Initializing task scheduler. 
(kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,302] INFO [ThrottledChannelReaper-Fetch]: Starting 
(kafka.server.ClientQuotaManager$ThrottledChannelReaper)
 [2021-08-26 10:21:06,303] INFO [ThrottledChannelReaper-Produce]: Starting 
(kafka.server.ClientQuotaManager$ThrottledChannelReaper)
 [2021-08-26 10:21:06,305] INFO [ThrottledChannelReaper-Request]: Starting 
(kafka.server.ClientQuotaManager$ThrottledChannelReaper)
 [2021-08-26 10:21:06,306] INFO [ThrottledChannelReaper-ControllerMutation]: 
Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
 [2021-08-26 10:21:06,333] INFO Loading logs from log dirs 
ArrayBuffer(/bitnami/kafka/data) (kafka.log.LogManager)
 [2021-08-26 10:21:06,335] INFO Skipping recovery for all logs in 
/bitnami/kafka/data since clean shutdown file was found (kafka.log.LogManager)
 [2021-08-26 10:21:06,342] DEBUG Loading log /bitnami/kafka/data/output-0 
(kafka.log.LogManager)
 [2021-08-26 10:21:06,379] DEBUG Loaded index file 
/bitnami/kafka/data/output-0/00000000000000000000.index with maxEntries = 273, 
maxIndexSize = 10485760, entries = 273, lastOffset = 17418, file position = 
2184 (kafka.log.OffsetIndex)
 [2021-08-26 10:21:06,385] DEBUG Resized 
/bitnami/kafka/data/output-0/00000000000000000000.index to 10485760, position 
is 2184 and limit is 10485760 (kafka.log.AbstractIndex)
 [2021-08-26 10:21:06,387] DEBUG Loaded index file 
/bitnami/kafka/data/output-0/00000000000000000000.timeindex with maxEntries = 
357, maxIndexSize = 10485760, entries = 357, lastOffset = 
TimestampOffset(1629973259582,17452), file position = 4284 (kafka.log.TimeIndex)
 [2021-08-26 10:21:06,387] DEBUG Resized 
/bitnami/kafka/data/output-0/00000000000000000000.timeindex to 10485756, 
position is 4284 and limit is 10485756 (kafka.log.AbstractIndex)
 [2021-08-26 10:21:06,399] DEBUG [LeaderEpochCache output-0] Cleared entries 
ListBuffer(EpochEntry(epoch=0, startOffset=0)) and rewrote first entry 
EpochEntry(epoch=0, startOffset=0) after truncating to start offset 0, leaving 
1 in the cache. (kafka.server.epoch.LeaderEpochFileCache)
 [2021-08-26 10:21:06,403] INFO [Log partition=output-0, 
dir=/bitnami/kafka/data] Loading producer state till offset 17453 with message 
format version 2 (kafka.log.Log)
 [2021-08-26 10:21:06,404] INFO [ProducerStateManager partition=output-0] 
Loading producer state from snapshot file 
'SnapshotFile(/bitnami/kafka/data/output-0/00000000000000017453.snapshot,17453)'
 (kafka.log.ProducerStateManager)
 [2021-08-26 10:21:06,413] DEBUG Scheduling task 
PeriodicProducerExpirationCheck with initial delay 600000 ms and period 600000 
ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,416] INFO Completed load of 
Log(dir=/bitnami/kafka/data/output-0, topic=output, partition=0, 
highWatermark=0, lastStableOffset=0, logStartOffset=0, logEndOffset=17453) with 
1 segments in 70ms (1/1 loaded in /bitnami/kafka/data) (kafka.log.LogManager)
 [2021-08-26 10:21:06,419] INFO Loaded 1 logs in 86ms. (kafka.log.LogManager)
 [2021-08-26 10:21:06,419] INFO Starting log cleanup with a period of 1000 ms. 
(kafka.log.LogManager)
 [2021-08-26 10:21:06,420] DEBUG Scheduling task kafka-log-retention with 
initial delay 30000 ms and period 1000 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,420] INFO Starting log flusher with a default period of 
9223372036854775807 ms. (kafka.log.LogManager)
 [2021-08-26 10:21:06,421] DEBUG Scheduling task kafka-log-flusher with initial 
delay 30000 ms and period 9223372036854775807 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,421] DEBUG Scheduling task 
kafka-recovery-point-checkpoint with initial delay 30000 ms and period 60000 
ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,422] DEBUG Scheduling task 
kafka-log-start-offset-checkpoint with initial delay 30000 ms and period 60000 
ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,423] DEBUG Scheduling task kafka-delete-logs with initial 
delay 30000 ms and period -1 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,739] INFO Updated connection-accept-rate max connection 
creation rate to 2147483647 (kafka.network.ConnectionQuotas)
 [2021-08-26 10:21:06,741] INFO Awaiting socket connections on 0.0.0.0:9092. 
(kafka.network.Acceptor)
 [2021-08-26 10:21:06,768] INFO [SocketServer listenerType=ZK_BROKER, nodeId=1] 
Created data-plane acceptor and processors for endpoint : 
ListenerName(PLAINTEXT) (kafka.network.SocketServer)
 [2021-08-26 10:21:06,768] INFO Updated connection-accept-rate max connection 
creation rate to 2147483647 (kafka.network.ConnectionQuotas)
 [2021-08-26 10:21:06,769] INFO Awaiting socket connections on 0.0.0.0:9093. 
(kafka.network.Acceptor)
 [2021-08-26 10:21:06,775] INFO [SocketServer listenerType=ZK_BROKER, nodeId=1] 
Created data-plane acceptor and processors for endpoint : 
ListenerName(DOCKERHOST) (kafka.network.SocketServer)
 [2021-08-26 10:21:06,795] INFO [broker-1-to-controller-send-thread]: Starting 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:06,795] DEBUG [broker-1-to-controller-send-thread]: 
Controller isn't cached, looking for local metadata changes 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:06,796] DEBUG [broker-1-to-controller-send-thread]: No 
controller defined in metadata cache, retrying after backoff 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:06,809] INFO [ExpirationReaper-1-Produce]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:06,810] INFO [ExpirationReaper-1-Fetch]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:06,811] INFO [ExpirationReaper-1-DeleteRecords]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:06,812] INFO [ExpirationReaper-1-ElectLeader]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:06,820] DEBUG Scheduling task isr-expiration with initial 
delay 0 ms and period 15000 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,820] DEBUG Scheduling task 
shutdown-idle-replica-alter-log-dirs-thread with initial delay 0 ms and period 
10000 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,823] INFO [LogDirFailureHandler]: Starting 
(kafka.server.ReplicaManager$LogDirFailureHandler)
 [2021-08-26 10:21:06,857] INFO Creating /brokers/ids/1 (is it secure? false) 
(kafka.zk.KafkaZkClient)
 [2021-08-26 10:21:06,877] INFO Stat of the created znode at /brokers/ids/1 is: 
1793,1793,1629973266864,1629973266864,1,0,0,72134560867287128,253,0,1793
 (kafka.zk.KafkaZkClient)
 [2021-08-26 10:21:06,877] INFO Registered broker 1 at path /brokers/ids/1 with 
addresses: PLAINTEXT://kafka-1:9092,DOCKERHOST://127.0.0.1:9093, czxid (broker 
epoch): 1793 (kafka.zk.KafkaZkClient)
 [2021-08-26 10:21:06,899] DEBUG [broker-1-to-controller-send-thread]: 
Controller isn't cached, looking for local metadata changes 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:06,899] DEBUG [broker-1-to-controller-send-thread]: No 
controller defined in metadata cache, retrying after backoff 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:06,924] INFO [ExpirationReaper-1-topic]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:06,927] DEBUG Try to create /controller and increment 
controller epoch to 89 with expected controller epoch zkVersion 88 
(kafka.zk.KafkaZkClient)
 [2021-08-26 10:21:06,928] INFO [ExpirationReaper-1-Heartbeat]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:06,929] INFO [ExpirationReaper-1-Rebalance]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:06,932] DEBUG [ZooKeeperClient Kafka server] Received event: 
WatchedEvent state:SyncConnected type:NodeCreated path:/controller 
(kafka.zookeeper.ZooKeeperClient)
 [2021-08-26 10:21:06,942] INFO [GroupCoordinator 1]: Starting up. 
(kafka.coordinator.group.GroupCoordinator)
 [2021-08-26 10:21:06,945] DEBUG Initializing task scheduler. 
(kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,945] DEBUG Scheduling task delete-expired-group-metadata 
with initial delay 0 ms and period 600000 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,946] INFO [GroupCoordinator 1]: Startup complete. 
(kafka.coordinator.group.GroupCoordinator)
 [2021-08-26 10:21:06,960] DEBUG [ProducerId Manager 1]: Read current 
producerId block 
(brokerId:1,blockStartProducerId:87000,blockEndProducerId:87999), Zk path 
version 88 (kafka.coordinator.transaction.ProducerIdManager)
 [2021-08-26 10:21:06,967] DEBUG Conditional update of path 
/latest_producer_id_block with value
Unknown macro: \{"version"}
and expected version 88 succeeded, returning the new version: 89 
(kafka.zk.KafkaZkClient)
 [2021-08-26 10:21:06,968] INFO [ProducerId Manager 1]: Acquired new producerId 
block (brokerId:1,blockStartProducerId:88000,blockEndProducerId:88999) by 
writing to Zk with path version 89 
(kafka.coordinator.transaction.ProducerIdManager)
 [2021-08-26 10:21:06,968] INFO [TransactionCoordinator id=1] Starting up. 
(kafka.coordinator.transaction.TransactionCoordinator)
 [2021-08-26 10:21:06,968] DEBUG Initializing task scheduler. 
(kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,969] DEBUG Scheduling task transaction-abort with initial 
delay 10000 ms and period 10000 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,971] DEBUG Scheduling task transactionalId-expiration 
with initial delay 3600000 ms and period 3600000 ms. 
(kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:06,972] INFO [Transaction Marker Channel Manager 1]: 
Starting (kafka.coordinator.transaction.TransactionMarkerChannelManager)
 [2021-08-26 10:21:06,972] INFO [TransactionCoordinator id=1] Startup complete. 
(kafka.coordinator.transaction.TransactionCoordinator)
 [2021-08-26 10:21:06,998] DEBUG [Controller id=1, targetBrokerId=1] Initiating 
connection to node kafka-1:9092 (id: 1 rack: null) using address 
kafka-1/172.18.0.3 (org.apache.kafka.clients.NetworkClient)
 [2021-08-26 10:21:07,000] INFO [ExpirationReaper-1-AlterAcls]: Starting 
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
 [2021-08-26 10:21:07,000] DEBUG [broker-1-to-controller-send-thread]: 
Controller isn't cached, looking for local metadata changes 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:07,000] DEBUG [broker-1-to-controller-send-thread]: No 
controller defined in metadata cache, retrying after backoff 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:07,003] DEBUG [Controller id=1, targetBrokerId=1] Created 
socket with SO_RCVBUF = 65536, SO_SNDBUF = 1313280, SO_TIMEOUT = 0 to node 1 
(org.apache.kafka.common.network.Selector)
 [2021-08-26 10:21:07,003] DEBUG [Controller id=1, targetBrokerId=1] Completed 
connection to node 1. Ready. (org.apache.kafka.clients.NetworkClient)
 [2021-08-26 10:21:07,025] DEBUG [Controller id=1, targetBrokerId=1] Sending 
UPDATE_METADATA request with header RequestHeader(apiKey=UPDATE_METADATA, 
apiVersion=7, clientId=1, correlationId=0) and timeout 30000 to node 1: 
UpdateMetadataRequestData(controllerId=1, controllerEpoch=89, brokerEpoch=1793, 
ungroupedPartitionStates=[], topicStates=[], 
liveBrokers=[UpdateMetadataBroker(id=1, v0Host='', v0Port=0, 
endpoints=[UpdateMetadataEndpoint(port=9092, host='kafka-1', 
listener='PLAINTEXT', securityProtocol=0), UpdateMetadataEndpoint(port=9093, 
host='127.0.0.1', listener='DOCKERHOST', securityProtocol=0)], rack=null)]) 
(org.apache.kafka.clients.NetworkClient)
 [2021-08-26 10:21:07,026] INFO [/config/changes-event-process-thread]: 
Starting 
(kafka.common.ZkNodeChangeNotificationListener$ChangeEventProcessThread)
 [2021-08-26 10:21:07,033] DEBUG Removing leader.replication.throttled.replicas 
from broker 1 for topic output (kafka.server.TopicConfigHandler)
 [2021-08-26 10:21:07,034] DEBUG Removing 
follower.replication.throttled.replicas from broker 1 for topic output 
(kafka.server.TopicConfigHandler)
 [2021-08-26 10:21:07,038] DEBUG Initializing task scheduler. 
(kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:07,038] DEBUG Scheduling task auto-leader-rebalance-task 
with initial delay 5000 ms and period -1000 ms. (kafka.utils.KafkaScheduler)
 [2021-08-26 10:21:07,041] INFO [SocketServer listenerType=ZK_BROKER, nodeId=1] 
Starting socket server acceptors and processors (kafka.network.SocketServer)
 [2021-08-26 10:21:07,042] DEBUG [SocketServer listenerType=ZK_BROKER, 
nodeId=1] Wait for authorizer to complete start up on listener 
ListenerName(PLAINTEXT) (kafka.network.SocketServer)
 [2021-08-26 10:21:07,043] DEBUG [SocketServer listenerType=ZK_BROKER, 
nodeId=1] Start processors on listener ListenerName(PLAINTEXT) 
(kafka.network.SocketServer)
 [2021-08-26 10:21:07,045] DEBUG [SocketServer listenerType=ZK_BROKER, 
nodeId=1] Start acceptor thread on listener ListenerName(PLAINTEXT) 
(kafka.network.SocketServer)
 [2021-08-26 10:21:07,046] INFO [SocketServer listenerType=ZK_BROKER, nodeId=1] 
Started data-plane acceptor and processor(s) for endpoint : 
ListenerName(PLAINTEXT) (kafka.network.SocketServer)
 [2021-08-26 10:21:07,046] DEBUG [SocketServer listenerType=ZK_BROKER, 
nodeId=1] Wait for authorizer to complete start up on listener 
ListenerName(DOCKERHOST) (kafka.network.SocketServer)
 [2021-08-26 10:21:07,046] DEBUG [SocketServer listenerType=ZK_BROKER, 
nodeId=1] Start processors on listener ListenerName(DOCKERHOST) 
(kafka.network.SocketServer)
 [2021-08-26 10:21:07,047] DEBUG [SocketServer listenerType=ZK_BROKER, 
nodeId=1] Start acceptor thread on listener ListenerName(DOCKERHOST) 
(kafka.network.SocketServer)
 [2021-08-26 10:21:07,048] INFO [SocketServer listenerType=ZK_BROKER, nodeId=1] 
Started data-plane acceptor and processor(s) for endpoint : 
ListenerName(DOCKERHOST) (kafka.network.SocketServer)
 [2021-08-26 10:21:07,049] INFO [SocketServer listenerType=ZK_BROKER, nodeId=1] 
Started socket server acceptors and processors (kafka.network.SocketServer)
 [2021-08-26 10:21:07,052] DEBUG Accepted connection from /172.18.0.1:46152 on 
/172.18.0.3:9093 and assigned it to processor 3, sendBufferSize 
[actual|requested]: [102400|102400] recvBufferSize [actual|requested]: 
[102400|102400] (kafka.network.Acceptor)
 [2021-08-26 10:21:07,052] DEBUG Accepted connection from /172.18.0.3:59924 on 
/172.18.0.3:9092 and assigned it to processor 0, sendBufferSize 
[actual|requested]: [102400|102400] recvBufferSize [actual|requested]: 
[102400|102400] (kafka.network.Acceptor)
 [2021-08-26 10:21:07,055] DEBUG Processor 0 listening to new connection from 
/172.18.0.3:59924 (kafka.network.Processor)
 [2021-08-26 10:21:07,055] DEBUG Processor 3 listening to new connection from 
/172.18.0.1:46152 (kafka.network.Processor)
 [2021-08-26 10:21:07,055] INFO Kafka version: 2.8.0 
(org.apache.kafka.common.utils.AppInfoParser)
 [2021-08-26 10:21:07,056] INFO Kafka commitId: ebb1d6e21cc92130 
(org.apache.kafka.common.utils.AppInfoParser)
 [2021-08-26 10:21:07,056] INFO Kafka startTimeMs: 1629973267049 
(org.apache.kafka.common.utils.AppInfoParser)
 [2021-08-26 10:21:07,058] INFO [KafkaServer id=1] started 
(kafka.server.KafkaServer)
 [2021-08-26 10:21:07,078] DEBUG [Controller id=1, targetBrokerId=1] Received 
UPDATE_METADATA response from node 1 for request with header 
RequestHeader(apiKey=UPDATE_METADATA, apiVersion=7, clientId=1, 
correlationId=0): UpdateMetadataResponseData(errorCode=0) 
(org.apache.kafka.clients.NetworkClient)
 [2021-08-26 10:21:07,082] DEBUG [Controller id=1, targetBrokerId=1] Sending 
LEADER_AND_ISR request with header RequestHeader(apiKey=LEADER_AND_ISR, 
apiVersion=5, clientId=1, correlationId=1) and timeout 30000 to node 1: 
LeaderAndIsrRequestData(controllerId=1, controllerEpoch=89, brokerEpoch=1793, 
type=0, ungroupedPartitionStates=[], 
topicStates=[LeaderAndIsrTopicState(topicName='output', 
topicId=te1YAUx9SiikZgwJoypbWg, 
partitionStates=[LeaderAndIsrPartitionState(topicName='output', 
partitionIndex=0, controllerEpoch=1, leader=1, leaderEpoch=0, isr=[1], 
zkVersion=0, replicas=[1], addingReplicas=[], removingReplicas=[], 
isNew=false)])], liveLeaders=[LeaderAndIsrLiveLeader(brokerId=1, 
hostName='kafka-1', port=9092)]) (org.apache.kafka.clients.NetworkClient)
 [2021-08-26 10:21:07,100] DEBUG [broker-1-to-controller-send-thread]: 
Controller isn't cached, looking for local metadata changes 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:07,101] INFO [broker-1-to-controller-send-thread]: Recorded 
new controller, from now on will use broker kafka-1:9092 (id: 1 rack: null) 
(kafka.server.BrokerToControllerRequestThread)
 [2021-08-26 10:21:07,102] INFO [ReplicaFetcherManager on broker 1] Removed 
fetcher for partitions Set(output-0) (kafka.server.ReplicaFetcherManager)
 [2021-08-26 10:21:07,104] DEBUG [KafkaApi-1] Produce request with correlation 
id 123 from client rdkafka on partition output-0 failed due to 
org.apache.kafka.common.errors.UnknownTopicOrPartitionException 
(kafka.server.KafkaApis)
 [2021-08-26 10:21:07,108] DEBUG [SocketServer listenerType=ZK_BROKER, 
nodeId=1] Connection with /172.18.0.1 disconnected 
(org.apache.kafka.common.network.Selector)
 java.io.EOFException
 at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:97)
 at org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:452)
 at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:402)
 at org.apache.kafka.common.network.Selector.attemptRead(Selector.java:674)
 at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:576)
 at org.apache.kafka.common.network.Selector.poll(Selector.java:481)
 at kafka.network.Processor.poll(SocketServer.scala:989)
 at kafka.network.Processor.run(SocketServer.scala:892)
 at java.base/java.lang.Thread.run(Thread.java:829)
 [2021-08-26 10:21:07,111] INFO [Partition output-0 broker=1] Log loaded for 
partition output-0 with initial high watermark 17453 (kafka.cluster.Partition)
 [2021-08-26 10:21:07,116] DEBUG Scheduling task highwatermark-checkpoint with 
initial delay 0 ms and period 5000 ms. (kafka.utils.KafkaScheduler)
{panel}

Configuration:
      - KAFKA_BROKER_ID=1
      - ALLOW_PLAINTEXT_LISTENER=yes
      - KAFKA_CFG_ZOOKEEPER_CONNECT=${KAFKA_ZOOKEEPER_CONNECT:-zookeeper:2181}
      - KAFKA_CFG_LISTENERS=PLAINTEXT://0.0.0.0:9092,DOCKERHOST://0.0.0.0:9093
      - 
KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,DOCKERHOST:PLAINTEXT
      - 
KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://kafka-1:9092,DOCKERHOST://127.0.0.1:9093
      - KAFKA_CFG_AUTO_CREATE_TOPICS_ENABLE=false
      - KAFKA_CFG_TOPIC_METADATA_REFRESH_INTERVAL_MS=10
      - KAFKA_CFG_METADATA_REFRESH_INTERVAL_MS=10
      - KAFKA_CFG_LOG_RETENTION_CHECK_INTERVAL_MS=1000
      - KAFKA_CFG_LOG_CLEANER_BACKOFF_MS=1000



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

Reply via email to