Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

rdkafka crashes when topic is too long (255 characters) sometimes #529

Closed
trthulhu opened this issue Feb 5, 2016 · 7 comments
Closed

rdkafka crashes when topic is too long (255 characters) sometimes #529

trthulhu opened this issue Feb 5, 2016 · 7 comments

Comments

@trthulhu
Copy link

trthulhu commented Feb 5, 2016

We're using rdkafka for (legacy) consuming data from Kafka. We sometimes run into this trace when we are testing against a long/random topic:

(gdb) where
#0 0x00007ffc3d007925 in raise () from /lib64/libc.so.6
#1 0x00007ffc3d009105 in abort () from /lib64/libc.so.6
#2 0x00007ffc1c4c9193 in rd_kafka_crash (file=Unhandled dwarf expression opcode 0xf3

) at rdkafka.c:2096
#3 0x00007ffc1c501cd3 in rd_kafka_toppar_remove (rktp=0x7ffb503161d0) at rdkafka_partition.c:165
#4 0x00007ffc1c501cf2 in rd_kafka_toppar_destroy_final (rktp=0x7ffb503161d0) at rdkafka_partition.c:183
#5 0x00007ffc1c4e9e7c in rd_kafka_op_destroy (rko=0x7ffb503a99a0) at rdkafka_op.c:107
#6 0x00007ffc1c4f15ce in rd_kafka_toppar_handle_Offset (rkb=Unhandled dwarf expression opcode 0xf3

) at rdkafka_request.c:245
#7 0x00007ffc1c4e5c21 in rd_kafka_buf_callback (rkb=0x7ffb500ea1c0, err=RD_KAFKA_RESP_ERR_NO_ERROR, response=0x7ffb50060ff0, request=0x7ffb503a7d00) at rdkafka_buf.c:472
#8 0x00007ffc1c4d5973 in rd_kafka_req_response (rkb=0x7ffb500ea1c0) at rdkafka_broker.c:913
#9 rd_kafka_recv (rkb=0x7ffb500ea1c0) at rdkafka_broker.c:1084
#10 0x00007ffc1c4e3940 in rd_kafka_transport_io_event (rktrans=0x7ffb50103400, events=1) at rdkafka_transport.c:1027
#11 0x00007ffc1c4d8a50 in rd_kafka_broker_serve (rkb=0x7ffb500ea1c0, timeout_ms=Unhandled dwarf expression opcode 0xf3

) at rdkafka_broker.c:2142
#12 0x00007ffc1c4d9679 in rd_kafka_broker_consumer_serve (arg=0x7ffb500ea1c0) at rdkafka_broker.c:3258
#13 rd_kafka_broker_thread_main (arg=0x7ffb500ea1c0) at rdkafka_broker.c:3366
#14 0x00007ffc1c50a797 in _thrd_wrapper_function (aArg=Unhandled dwarf expression opcode 0xf3

) at tinycthread.c:611
#15 0x00007ffc3cbbb9d1 in start_thread () from /lib64/libpthread.so.0
#16 0x00007ffc3d0bdb5d in clone () from /lib64/libc.so.6

The topic name is: "s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo"

@edenhill
Copy link
Contributor

edenhill commented Feb 6, 2016

Reproduced, trying to work out what's going on.
It seems like the broker's topic state is messed up, both kafka-topics.sh --describe (which queries zookeeper) and kafkacat -L (which queries the broker) shows that each partition of the topic has a leader, but producing to the topic fails with "Not leader for partition".
Verified both with kafka-console-producer.sh and kafkacat:
[2016-02-06 17:20:51,109] INFO [KafkaApi-2] Closing connection due to error during produce request with correlation id 1 from client id console-producer with ack=0 Topic and partition to exceptions: [s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo,2] -> org.apache.kafka.common.errors.NotLeaderForPartitionException (kafka.server.KafkaApis)

There are also some exceptions in the zk log:
[2016-02-06 12:24:52,358] INFO Got user-level KeeperException when processing sessionid:0x152828fa18c0001 type:setData cxid:0xac47 zxid:0xb1a76 txntype:-1 reqpath:n/a Error Path:/config/topics/s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo Error:KeeperErrorCode = NoNode for /config/topics/s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo (org.apache.zookeeper.server.PrepRequestProcessor)
followed by similar errors for type:create.

kafka.topics.sh --describe output:
Topic:s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo PartitionCount:4 ReplicationFactor:3 Configs: Topic: s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo Partition: 0 Leader: 3 Replicas: 3,1,2 Isr: 3,1,2 Topic: s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo Partition: 1 Leader: 1 Replicas: 1,2,3 Isr: 1,2,3 Topic: s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo Partition: 2 Leader: 2 Replicas: 2,3,1 Isr: 2,3,1 Topic: s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo Partition: 3 Leader: 3 Replicas: 3,2,1 Isr: 3,2,1

kafkacat -L output:
1 topics: topic "s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo" with 4 partitions: partition 2, leader 2, replicas: 2,3,1, isrs: 2,3,1 partition 1, leader 1, replicas: 1,2,3, isrs: 1,2,3 partition 3, leader 3, replicas: 3,2,1, isrs: 3,2,1 partition 0, leader 3, replicas: 3,1,2, isrs: 3,1,2

Producing from kafkacat seems to hang, but enabling 'msg' debug shows that the produce request fails and is retried:
%7|1454776375.126|MSGSET|rdkafka#producer-1| eden:9093/3: MessageSet with 1 message(s) encountered error: Broker: Not leader for partition

Producing with kafka-console-producer.sh does not give any errors, but the message is not delivered.

@edenhill
Copy link
Contributor

edenhill commented Feb 6, 2016

The crash is clearly a librdkafka bug though.

@fpj
Copy link

fpj commented Feb 6, 2016

@edenhill to double check, I have created with the CLI and it works fine, see the output below:

[zk: localhost:2181(CONNECTED) 0] create /config
Created /config
[zk: localhost:2181(CONNECTED) 1] create /config/topics
Created /config/topics
[zk: localhost:2181(CONNECTED) 2] create /config/topics/s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo test
Created /config/topics/s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo
[zk: localhost:2181(CONNECTED) 3] get /config/topics/s_bwdMQqvcnnOyypsNaikzSyWIrGxjAsj_lvwhdzLGDppr_wQYBJEfxRUkzVbVbTq_DbB_uDNO_nbTqJfwSUAgREeIaeawU_JbYaPgaFimHBJHwXoxWnfYYxYRhUEXuuiUKVU__omqjamNqvovvsaLhOTsXhTsYfodeKOwUsXLWNznWVHxuzIWDNukjfIemmkvfVUbAtI_mMvqFNuMWGniKdfMxUMzzsPHrZEHAqeeQYNNcQbxxMpQwxKHw_Bjo
test
[zk: localhost:2181(CONNECTED) 4]

I have also verified with other longer paths.

@edenhill
Copy link
Contributor

edenhill commented Feb 6, 2016

Seems like the broker doesn't like topics of 254 chars or more:

TOPIC=$(printf 'd%.0s' {1..254} ) ; bin/kafka-topics.sh --zookeeper 0 --create --topic $TOPIC --partitions 1 --replication-factor 1

[2016-02-06 22:00:01,943] INFO [ReplicaFetcherManager on broker 3] Removed fetcher for partitions [dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd,0] (kafka.server.ReplicaFetcherManager)
[2016-02-06 22:00:01,944] ERROR [KafkaApi-3] Error when handling request {controller_id=3,controller_epoch=12,partition_states=[{topic=dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd,partition=0,controller_epoch=12,leader=3,leader_epoch=0,isr=[3],zk_version=0,replicas=[3]}],live_leaders=[{id=3,host=eden,port=9093}]} (kafka.server.KafkaApis)
java.lang.NullPointerException
        at scala.collection.mutable.ArrayOps$ofRef$.length$extension(ArrayOps.scala:114)
        at scala.collection.mutable.ArrayOps$ofRef.length(ArrayOps.scala:114)
        at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:32)
        at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:108)
        at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
        at kafka.log.Log.loadSegments(Log.scala:138)
        at kafka.log.Log.<init>(Log.scala:92)
        at kafka.log.LogManager.createLog(LogManager.scala:357)
        at kafka.cluster.Partition.getOrCreateReplica(Partition.scala:96)
        at kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:176)
        at kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:176)
        at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
        at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:176)
        at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:170)
        at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:259)
        at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:267)
        at kafka.cluster.Partition.makeLeader(Partition.scala:170)
        at kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:696)
        at kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:695)
        at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
        at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
        at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
        at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
        at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
        at kafka.server.ReplicaManager.makeLeaders(ReplicaManager.scala:695)
        at kafka.server.ReplicaManager.becomeLeaderOrFollower(ReplicaManager.scala:641)
        at kafka.server.KafkaApis.handleLeaderAndIsrRequest(KafkaApis.scala:142)
        at kafka.server.KafkaApis.handle(KafkaApis.scala:79)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
        at java.lang.Thread.run(Thread.java:745)

@edenhill
Copy link
Contributor

edenhill commented Feb 8, 2016

The broker state bug only happens when these >=254 topics are created with kafka-topics.sh --create, not by topic auto creation.

@edenhill
Copy link
Contributor

edenhill commented Feb 8, 2016

Created upstream issue: https://issues.apache.org/jira/browse/KAFKA-3219

edenhill added a commit that referenced this issue Feb 11, 2016
This is a 0.9.0 broker bug which lead to some problems in librdkafka,
so while this test will still fail on broker 0.9.0 the crash in
librdkafka will be caught.
@edenhill
Copy link
Contributor

Crash is fixed, please verify on your end too.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

3 participants