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

Yannick commented on KAFKA-9212:
--------------------------------

Yeah the topic is correctly replicated according metadata output from tools 
like kafkacat :

 

As of today, we downgrade our clients to 2.2.1 to avoid being stuck in this 
fencing loop ( 2.3 client handle the FENCED_LEADER_EPOCH ).

We restarted the 3 brokers ( rolling restart) and still have discrepancies 
between those checkpoint files as follows :

 

Broker ID 4 :

cat /var/lib/kafka/logs/connect_ls_config-0/leader-epoch-checkpoint
0
2
0 0
6 22

 

Broker ID 1 :

cat /var/lib/kafka/logs/connect_ls_config-0/leader-epoch-checkpoint
0
2
0 0
5 22

 

Broker ID 3:

cat /var/lib/kafka/logs/connect_ls_config-0/leader-epoch-checkpoint
0
1
0 0

 

 

Regarding the dump of this topic, here they are ( there is just one .log file . 
for all brokers) ( cannot show the content using print-data-log as it might 
contain sensitive info) :

 

Broker ID 1 :

/opt/kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files 
/var/lib/kafka/logs/connect_ls_config-0/00000000000000000000.log
Dumping /var/lib/kafka/logs/connect_ls_config-0/00000000000000000000.log
Starting offset: 0
baseOffset: 0 lastOffset: 0 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 0 CreateTime: 1573660711038 size: 962 magic: 2 
compresscodec: NONE crc: 1786879997 isvalid: true
baseOffset: 1 lastOffset: 1 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 962 CreateTime: 1573660712089 size: 1009 magic: 2 
compresscodec: NONE crc: 1230182444 isvalid: true
baseOffset: 2 lastOffset: 3 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 1971 CreateTime: 1573660712091 size: 1957 magic: 2 
compresscodec: NONE crc: 2419651795 isvalid: true
baseOffset: 4 lastOffset: 4 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 3928 CreateTime: 1573660712611 size: 89 magic: 2 
compresscodec: NONE crc: 3321423372 isvalid: true
baseOffset: 5 lastOffset: 5 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 4017 CreateTime: 1573751698440 size: 962 magic: 2 
compresscodec: NONE crc: 704355531 isvalid: true
baseOffset: 6 lastOffset: 6 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 4979 CreateTime: 1573751699462 size: 1009 magic: 2 
compresscodec: NONE crc: 1489459952 isvalid: true
baseOffset: 7 lastOffset: 8 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 5988 CreateTime: 1573751699463 size: 1957 magic: 2 
compresscodec: NONE crc: 657348671 isvalid: true
baseOffset: 9 lastOffset: 9 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 7945 CreateTime: 1573751699985 size: 89 magic: 2 
compresscodec: NONE crc: 1825092385 isvalid: true
baseOffset: 10 lastOffset: 11 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 8034 CreateTime: 1573828311242 size: 104 magic: 2 
compresscodec: NONE crc: 3533917687 isvalid: true
baseOffset: 12 lastOffset: 12 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 8138 CreateTime: 1573828467292 size: 953 magic: 2 
compresscodec: NONE crc: 232359935 isvalid: true
baseOffset: 13 lastOffset: 13 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 9091 CreateTime: 1573828467807 size: 1000 magic: 2 
compresscodec: NONE crc: 1484213287 isvalid: true
baseOffset: 14 lastOffset: 15 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 10091 CreateTime: 1573828467808 size: 1939 magic: 2 
compresscodec: NONE crc: 49865436 isvalid: true
baseOffset: 16 lastOffset: 16 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 12030 CreateTime: 1573828468331 size: 94 magic: 2 
compresscodec: NONE crc: 1480833250 isvalid: true
baseOffset: 17 lastOffset: 17 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 12124 CreateTime: 1573828530715 size: 986 magic: 2 
compresscodec: NONE crc: 678439265 isvalid: true
baseOffset: 18 lastOffset: 19 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 13110 CreateTime: 1573828531239 size: 2005 magic: 2 
compresscodec: NONE crc: 1542429159 isvalid: true
baseOffset: 20 lastOffset: 20 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 15115 CreateTime: 1573828531239 size: 1033 magic: 2 
compresscodec: NONE crc: 865245135 isvalid: true
baseOffset: 21 lastOffset: 21 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 16148 CreateTime: 1573828531761 size: 101 magic: 2 
compresscodec: NONE crc: 4023495638 isvalid: true

 

 

Broker ID 3 :

 

~$ /opt/kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files 
/var/lib/kafka/logs/connect_ls_config-0/00000000000000000000.log
Dumping /var/lib/kafka/logs/connect_ls_config-0/00000000000000000000.log
Starting offset: 0
baseOffset: 0 lastOffset: 0 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 0 CreateTime: 1573660711038 size: 962 magic: 2 
compresscodec: NONE crc: 1786879997 isvalid: true
baseOffset: 1 lastOffset: 1 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 962 CreateTime: 1573660712089 size: 1009 magic: 2 
compresscodec: NONE crc: 1230182444 isvalid: true
baseOffset: 2 lastOffset: 3 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 1971 CreateTime: 1573660712091 size: 1957 magic: 2 
compresscodec: NONE crc: 2419651795 isvalid: true
baseOffset: 4 lastOffset: 4 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 3928 CreateTime: 1573660712611 size: 89 magic: 2 
compresscodec: NONE crc: 3321423372 isvalid: true
baseOffset: 5 lastOffset: 5 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 4017 CreateTime: 1573751698440 size: 962 magic: 2 
compresscodec: NONE crc: 704355531 isvalid: true
baseOffset: 6 lastOffset: 6 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 4979 CreateTime: 1573751699462 size: 1009 magic: 2 
compresscodec: NONE crc: 1489459952 isvalid: true
baseOffset: 7 lastOffset: 8 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 5988 CreateTime: 1573751699463 size: 1957 magic: 2 
compresscodec: NONE crc: 657348671 isvalid: true
baseOffset: 9 lastOffset: 9 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 7945 CreateTime: 1573751699985 size: 89 magic: 2 
compresscodec: NONE crc: 1825092385 isvalid: true
baseOffset: 10 lastOffset: 11 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 8034 CreateTime: 1573828311242 size: 104 magic: 2 
compresscodec: NONE crc: 3533917687 isvalid: true
baseOffset: 12 lastOffset: 12 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 8138 CreateTime: 1573828467292 size: 953 magic: 2 
compresscodec: NONE crc: 232359935 isvalid: true
baseOffset: 13 lastOffset: 13 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 9091 CreateTime: 1573828467807 size: 1000 magic: 2 
compresscodec: NONE crc: 1484213287 isvalid: true
baseOffset: 14 lastOffset: 15 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 10091 CreateTime: 1573828467808 size: 1939 magic: 2 
compresscodec: NONE crc: 49865436 isvalid: true
baseOffset: 16 lastOffset: 16 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 12030 CreateTime: 1573828468331 size: 94 magic: 2 
compresscodec: NONE crc: 1480833250 isvalid: true
baseOffset: 17 lastOffset: 17 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 12124 CreateTime: 1573828530715 size: 986 magic: 2 
compresscodec: NONE crc: 678439265 isvalid: true
baseOffset: 18 lastOffset: 19 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 13110 CreateTime: 1573828531239 size: 2005 magic: 2 
compresscodec: NONE crc: 1542429159 isvalid: true
baseOffset: 20 lastOffset: 20 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 15115 CreateTime: 1573828531239 size: 1033 magic: 2 
compresscodec: NONE crc: 865245135 isvalid: true
baseOffset: 21 lastOffset: 21 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 16148 CreateTime: 1573828531761 size: 101 magic: 2 
compresscodec: NONE crc: 4023495638 isvalid: true

 

 

Broker ID 4 : 

 

Dumping /var/lib/kafka/logs/connect_ls_config-0/00000000000000000000.log
Starting offset: 0
baseOffset: 0 lastOffset: 0 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 0 CreateTime: 1573660711038 size: 962 magic: 2 
compresscodec: NONE crc: 1786879997 isvalid: true
baseOffset: 1 lastOffset: 1 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 962 CreateTime: 1573660712089 size: 1009 magic: 2 
compresscodec: NONE crc: 1230182444 isvalid: true
baseOffset: 2 lastOffset: 3 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 1971 CreateTime: 1573660712091 size: 1957 magic: 2 
compresscodec: NONE crc: 2419651795 isvalid: true
baseOffset: 4 lastOffset: 4 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 3928 CreateTime: 1573660712611 size: 89 magic: 2 
compresscodec: NONE crc: 3321423372 isvalid: true
baseOffset: 5 lastOffset: 5 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 4017 CreateTime: 1573751698440 size: 962 magic: 2 
compresscodec: NONE crc: 704355531 isvalid: true
baseOffset: 6 lastOffset: 6 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 4979 CreateTime: 1573751699462 size: 1009 magic: 2 
compresscodec: NONE crc: 1489459952 isvalid: true
baseOffset: 7 lastOffset: 8 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 5988 CreateTime: 1573751699463 size: 1957 magic: 2 
compresscodec: NONE crc: 657348671 isvalid: true
baseOffset: 9 lastOffset: 9 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 7945 CreateTime: 1573751699985 size: 89 magic: 2 
compresscodec: NONE crc: 1825092385 isvalid: true
baseOffset: 10 lastOffset: 11 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 8034 CreateTime: 1573828311242 size: 104 magic: 2 
compresscodec: NONE crc: 3533917687 isvalid: true
baseOffset: 12 lastOffset: 12 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 8138 CreateTime: 1573828467292 size: 953 magic: 2 
compresscodec: NONE crc: 232359935 isvalid: true
baseOffset: 13 lastOffset: 13 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 9091 CreateTime: 1573828467807 size: 1000 magic: 2 
compresscodec: NONE crc: 1484213287 isvalid: true
baseOffset: 14 lastOffset: 15 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 10091 CreateTime: 1573828467808 size: 1939 magic: 2 
compresscodec: NONE crc: 49865436 isvalid: true
baseOffset: 16 lastOffset: 16 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 12030 CreateTime: 1573828468331 size: 94 magic: 2 
compresscodec: NONE crc: 1480833250 isvalid: true
baseOffset: 17 lastOffset: 17 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 12124 CreateTime: 1573828530715 size: 986 magic: 2 
compresscodec: NONE crc: 678439265 isvalid: true
baseOffset: 18 lastOffset: 19 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 13110 CreateTime: 1573828531239 size: 2005 magic: 2 
compresscodec: NONE crc: 1542429159 isvalid: true
baseOffset: 20 lastOffset: 20 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 15115 CreateTime: 1573828531239 size: 1033 magic: 2 
compresscodec: NONE crc: 865245135 isvalid: true
baseOffset: 21 lastOffset: 21 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false 
isControl: false position: 16148 CreateTime: 1573828531761 size: 101 magic: 2 
compresscodec: NONE crc: 4023495638 isvalid: true

 

 

 

Here is the conf used by all brokers : 

 

broker.id=XX

delete.topic.enable=true


listeners=INSIDE://0.0.0.0:9092,OUTSIDE://0.0.0.0:9492
advertised.listeners=REDACTED
listener.security.protocol.map=INSIDE:PLAINTEXT,OUTSIDE:SSL
inter.broker.listener.name=INSIDE
#SSL conf REDACTED

ssl.client.auth=required
num.network.threads=3

num.io.threads=8

socket.send.buffer.bytes=102400

socket.receive.buffer.bytes=102400

socket.request.max.bytes=104857600

inter.broker.protocol.version=2.3

 auto.create.topics.enable=true

log.dirs=/var/lib/kafka/logs

num.partitions=3

num.recovery.threads.per.data.dir=1

default.replication.factor=3

offsets.topic.replication.factor=3
transaction.state.log.replication.factor=3
transaction.state.log.min.isr=2

replica.socket.timeout.ms=30000

replica.fetch.wait.max.ms=5000

replica.lag.time.max.ms=15000

min.insync.replicas=2

acks=all

log.retention.hours=24


log.segment.bytes=1073741824

log.retention.check.interval.ms=300000

log.roll.hours=168


zookeeper.connect=REDACTED

zookeeper.connection.timeout.ms=6000

 

group.initial.rebalance.delay.ms=3000

transactional.id.expiration.ms=1814400000

offsets.retention.minutes=11580

 

 

> Keep receiving FENCED_LEADER_EPOCH while sending ListOffsetRequest
> ------------------------------------------------------------------
>
>                 Key: KAFKA-9212
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9212
>             Project: Kafka
>          Issue Type: Bug
>          Components: consumer, offset manager
>    Affects Versions: 2.3.0
>         Environment: Linux
>            Reporter: Yannick
>            Priority: Critical
>
> When running Kafka connect s3 sink connector ( confluent 5.3.0), after one 
> broker got restarted (leaderEpoch updated at this point), the connect worker 
> crashed with the following error : 
> [2019-11-19 16:20:30,097] ERROR [Worker clientId=connect-1, 
> groupId=connect-ls] Uncaught exception in herder work thread, exiting: 
> (org.apache.kafka.connect.runtime.distributed.DistributedHerder:253)
>  org.apache.kafka.common.errors.TimeoutException: Failed to get offsets by 
> times in 30003ms
>  
> After investigation, it seems it's because it got fenced when sending 
> ListOffsetRequest in loop and then got timed out , as follows :
> [2019-11-19 16:20:30,020] DEBUG [Consumer clientId=consumer-3, 
> groupId=connect-ls] Sending ListOffsetRequest (type=ListOffsetRequest, 
> replicaId=-1, partitionTimestamps={connect_ls_config-0={timestamp: -1, 
> maxNumOffsets: 1, currentLeaderEpoch: Optional[1]}}, 
> isolationLevel=READ_UNCOMMITTED) to broker kafka6.fra2.internal:9092 (id: 4 
> rack: null) (org.apache.kafka.clients.consumer.internals.Fetcher:905)
> [2019-11-19 16:20:30,044] DEBUG [Consumer clientId=consumer-3, 
> groupId=connect-ls] Attempt to fetch offsets for partition 
> connect_ls_config-0 failed due to FENCED_LEADER_EPOCH, retrying. 
> (org.apache.kafka.clients.consumer.internals.Fetcher:985)
>  
> The above happens multiple times until timeout.
>  
> According to the debugs, the consumer always get a leaderEpoch of 1 for this 
> topic when starting up :
>  
>  [2019-11-19 13:27:30,802] DEBUG [Consumer clientId=consumer-3, 
> groupId=connect-ls] Updating last seen epoch from null to 1 for partition 
> connect_ls_config-0 (org.apache.kafka.clients.Metadata:178)
>   
>   
>  But according to our brokers log, the leaderEpoch should be 2, as follows :
>   
>  [2019-11-18 14:19:28,988] INFO [Partition connect_ls_config-0 broker=4] 
> connect_ls_config-0 starts at Leader Epoch 2 from offset 22. Previous Leader 
> Epoch was: 1 (kafka.cluster.Partition)
>   
>   
>  This make impossible to restart the worker as it will always get fenced and 
> then finally timeout.
>   
>  It is also impossible to consume with a 2.3 kafka-console-consumer as 
> follows :
>   
>  kafka-console-consumer --bootstrap-server BOOTSTRAPSERVER:9092 --topic 
> connect_ls_config --from-beginning 
>   
>  the above will just hang forever ( which is not expected cause there is 
> data) and we can see those debug messages :
> [2019-11-19 22:17:59,124] DEBUG [Consumer clientId=consumer-1, 
> groupId=console-consumer-3844] Attempt to fetch offsets for partition 
> connect_ls_config-0 failed due to FENCED_LEADER_EPOCH, retrying. 
> (org.apache.kafka.clients.consumer.internals.Fetcher)
>   
>   
>  Interesting fact, if we do subscribe the same way with kafkacat (1.5.0) we 
> can consume without problem ( must be the way kafkacat is consuming ignoring 
> FENCED_LEADER_EPOCH):
>   
>  kafkacat -b BOOTSTRAPSERVER:9092 -t connect_ls_config -o beginning
>   
>   



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

Reply via email to