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

ASF GitHub Bot commented on KAFKA-4858:
---------------------------------------

GitHub user vahidhashemian opened a pull request:

    https://github.com/apache/kafka/pull/2673

    KAFKA-4858: Broker should ignore long topic names

    Starting from the 0.10.0.0 release topics names of over 249 characters are 
not allowed (`kafka-topics.sh` would disallow the creation). However, it is 
possible to use an older client version against the more recent brokers and 
attempt to create topics with longer names.
    This causes problems with the broker when trying to bring the corresponding 
partitions online (as described in the 
[JIRA](https://issues.apache.org/jira/browse/KAFKA-4858)).
    This PR makes the broker ignore those topics to avoid the reported failures.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/vahidhashemian/kafka KAFKA-4858

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/2673.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2673
    
----
commit 0614c9d0e59fc0b6f63b42b1358e8f0ed4f450dc
Author: Vahid Hashemian <vahidhashem...@us.ibm.com>
Date:   2017-03-10T22:44:47Z

    KAFKA-4858: Broker should ignore long topic names
    
    Starting from the 0.10.0.0 release topics names of over 249 characters are 
not allowed (`kafka-topics.sh` would disallow the creation). However, it is 
possible to use an older client version against the more recent brokers and 
attempt to create topics with longer names.
    This causes problems with the broker when trying to bring the corresponding 
partitions online (as described in the 
[JIRA](https://issues.apache.org/jira/browse/KAFKA-4858)).
    This PR makes the broker ignore those topics to avoid the reported failures.

----


> Long topic names created using old kafka-topics.sh can prevent newer brokers 
> from joining any ISRs
> --------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-4858
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4858
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.10.1.1, 0.10.2.0
>            Reporter: James Cheng
>            Assignee: Vahid Hashemian
>
> I ran into a variant of KAFKA-3219 that resulted in a broker being unable to 
> join any ISRs the cluster.
> Prior to 0.10.0.0, the maximum topic length was 255.
> With 0.10.0.0 and beyond, the maximum topic length is 249.
> The check on topic name length is done by kafka-topics.sh prior to topic 
> creation. Thus, it is possible to use a 0.9.0.1 kafka-topics.sh script to 
> create a 255 character topic on a 0.10.1.1 broker.
> When this happens, you will get the following stack trace (the same one seen 
> in KAFKA-3219)
> {code}
> $ TOPIC=$(printf 'd%.0s' {1..255} ) ; bin/kafka-topics.sh --zookeeper 
> 127.0.0.1 --create --topic $TOPIC --partitions 1 --replication-factor 2
> Created topic 
> "ddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd".
> {code}
> {code}
> [2017-03-06 22:01:19,011] ERROR [KafkaApi-2] Error when handling request 
> {controller_id=1,controller_epoch=1,partition_states=[{topic=ddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd,partition=0,controller_epoch=1,leader=2,leader_epoch=0,isr=[2,1],zk_version=0,replicas=[2,1]}],live_leaders=[{id=2,host=jchengmbpro15,port=9093}]}
>  (kafka.server.KafkaApis)
> java.lang.NullPointerException
>       at 
> scala.collection.mutable.ArrayOps$ofRef$.length$extension(ArrayOps.scala:192)
>       at scala.collection.mutable.ArrayOps$ofRef.length(ArrayOps.scala:192)
>       at 
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:32)
>       at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>       at 
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732)
>       at kafka.log.Log.loadSegments(Log.scala:155)
>       at kafka.log.Log.<init>(Log.scala:108)
>       at kafka.log.LogManager.createLog(LogManager.scala:362)
>       at kafka.cluster.Partition.getOrCreateReplica(Partition.scala:94)
>       at 
> kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
>       at 
> kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
>       at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
>       at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:174)
>       at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:168)
>       at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
>       at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:242)
>       at kafka.cluster.Partition.makeLeader(Partition.scala:168)
>       at 
> kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:758)
>       at 
> kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:757)
>       at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
>       at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
>       at 
> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
>       at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
>       at scala.collection.mutable.HashMap.foreach(HashMap.scala:99)
>       at kafka.server.ReplicaManager.makeLeaders(ReplicaManager.scala:757)
>       at 
> kafka.server.ReplicaManager.becomeLeaderOrFollower(ReplicaManager.scala:703)
>       at kafka.server.KafkaApis.handleLeaderAndIsrRequest(KafkaApis.scala:148)
>       at kafka.server.KafkaApis.handle(KafkaApis.scala:82)
>       at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
>       at java.lang.Thread.run(Thread.java:745)
> {code}
> The topic does not get created on disk, but the broker thinks the topic is 
> ready. The broker seems functional, for other topics. I can produce/consume 
> to other topics.
> {code}
> $ ./bin/kafka-topics.sh --zookeeper 127.0.0.1 --describe
> Topic:ddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd
>  PartitionCount:1        ReplicationFactor:2     Configs:
>       Topic: 
> ddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd
>   Partition: 0    Leader: 2       Replicas: 2,1   Isr: 2,1
> {code}
> If you stop and restart the broker, it again gets that stack trace. This 
> time, the broker fails to join *any* ISRs in the cluster. Notice below that 
> broker 2 is out of all ISRs
> {code}
> $ ./bin/kafka-topics.sh --zookeeper 127.0.0.1 --describe
> Topic:ddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd
>  PartitionCount:1        ReplicationFactor:2     Configs:
>       Topic: 
> ddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd
>   Partition: 0    Leader: 1       Replicas: 2,1   Isr: 1
> Topic:small   PartitionCount:5        ReplicationFactor:2     Configs:
>       Topic: small    Partition: 0    Leader: 1       Replicas: 1,2   Isr: 1
>       Topic: small    Partition: 1    Leader: 1       Replicas: 2,1   Isr: 1
>       Topic: small    Partition: 2    Leader: 1       Replicas: 1,2   Isr: 1
>       Topic: small    Partition: 3    Leader: 1       Replicas: 2,1   Isr: 1
>       Topic: small    Partition: 4    Leader: 1       Replicas: 1,2   Isr: 1
> {code}
> So, it appears that a long topic name that sneaks into the cluster can 
> prevent brokers from partipating in the cluster.
> Furthermore, I'm not exactly sure how to delete the offending topic. A 
> kafka-topics.sh --delete won't delete the topic because it can't talk to all 
> replicas, because the replicas are not in the ISR. We ran into this at work 
> today and ended up having to manually delete the topic configuration from 
> zookeeper and then did a bounce of all affected brokers. Until we did that, 
> those brokers weren't able to join the cluster.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to