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

Edoardo Comar commented on KAFKA-10220:
---------------------------------------

ok, a bit more steps to reproduce.... looks like the issue was linked to having 
created and described a topic with 2.5

{{#on current trunk}}
{{$ git lg}}
{{55b5b248c - (HEAD -> trunk, origin/trunk, origin/HEAD)}}

{{#build}}
{{$ ./gradlew clean}}
{{$ ./gradlew assemble -PscalaVersion=2.12}}

{{#run zookepeer and one broker}}
{{$ export SCALA_VERSION=2.12}}
{{$  bin/zookeeper-server-start.sh config/zookeeper.properties}}
{{$ bin/kafka-server-start.sh config/server.properties}}

{{#create topic with CLI}}
{{$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --create --topic 
topic1 --partitions 1 --replication-factor 1}}
{{# works fine}}
{{# describe topic with CLI}}
{{$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --describe}}
{{Topic: topic1 PartitionCount: 1 ReplicationFactor: 1 Configs: 
segment.bytes=1073741824Topic: topic1 PartitionCount: 1 ReplicationFactor: 1 
Configs: segment.bytes=1073741824 Topic: topic1 Partition: 0 Leader: 0 
Replicas: 0 Isr: 0}}
{{#works fine}}

{{in another terminal, use the kafka_2.13-2.5.0 binary distribution:}}
{{$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --create --topic 
topic25 --partitions 1 --replication-factor 1}}
{{Created topic topic25.}}
{{#now describe it (still with 2.5) ... boom}}
{{$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --describe}}
{{Error while executing topic command : 
org.apache.kafka.common.errors.UnknownServerException: The server experienced 
an unexpected error when processing the request.Error while executing topic 
command : org.apache.kafka.common.errors.UnknownServerException: The server 
experienced an unexpected error when processing the request.[2020-06-30 
12:39:48,581] ERROR java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.UnknownServerException: The server experienced 
an unexpected error when processing the request. at 
org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)}}

{{#in the broker terminal, the exception is }}

{{[2020-06-30 12:40:27,543] ERROR [Admin Manager on Broker 0]: Error processing 
describe configs request for resource DescribeConfigsResource(resourceType=2, 
resourceName='topic1', configurationKeys=null) 
(kafka.server.AdminManager)[2020-06-30 12:40:27,543] ERROR [Admin Manager on 
Broker 0]: Error processing describe configs request for resource 
DescribeConfigsResource(resourceType=2, resourceName='topic1', 
configurationKeys=null) 
(kafka.server.AdminManager)java.lang.NullPointerException at 
kafka.server.AdminManager.$anonfun$describeConfigs$3(AdminManager.scala:359) at 
kafka.server.AdminManager.$anonfun$describeConfigs$3$adapted(AdminManager.scala:357)
 at 
scala.collection.TraversableLike.$anonfun$filterImpl$1(TraversableLike.scala:291)
 at scala.collection.Iterator.foreach(Iterator.scala:943) at 
scala.collection.Iterator.foreach$(Iterator.scala:943) at 
scala.collection.AbstractIterator.foreach(Iterator.scala:1431) at 
scala.collection.IterableLike.foreach(IterableLike.scala:74) at 
scala.collection.IterableLike.foreach$(IterableLike.scala:73) at 
scala.collection.AbstractIterable.foreach(Iterable.scala:56) at 
scala.collection.TraversableLike.filterImpl(TraversableLike.scala:290) at 
scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:284) at 
scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108) at 
scala.collection.TraversableLike.filter(TraversableLike.scala:382) at 
scala.collection.TraversableLike.filter$(TraversableLike.scala:382) at 
scala.collection.AbstractTraversable.filter(Traversable.scala:108) at 
kafka.server.AdminManager.$anonfun$describeConfigs$1(AdminManager.scala:357) at 
kafka.server.AdminManager.describeConfigs(AdminManager.scala:350) at 
kafka.server.KafkaApis.handleDescribeConfigsRequest(KafkaApis.scala:2594) at 
kafka.server.KafkaApis.handle(KafkaApis.scala:165) at 
kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:70) at 
java.base/java.lang.Thread.run(Thread.java:834)}}

> NPE when describing resources
> -----------------------------
>
>                 Key: KAFKA-10220
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10220
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>            Reporter: Edoardo Comar
>            Priority: Major
>
> In current trunk code 
>  Describing a topic from the CLI can fail with an NPE in the broker
> on the line 
> {{          
> resource.configurationKeys.asScala.forall(_.contains(configName))}}
>  
> (configurationKeys is null)
> {{[2020-06-30 11:10:39,464] ERROR [Admin Manager on Broker 0]: Error 
> processing describe configs request for resource 
> DescribeConfigsResource(resourceType=2, resourceName='topic1', 
> configurationKeys=null) 
> (kafka.server.AdminManager)}}{{java.lang.NullPointerException}}{{at 
> kafka.server.AdminManager.$anonfun$describeConfigs$3(AdminManager.scala:395)}}{{at
>  
> kafka.server.AdminManager.$anonfun$describeConfigs$3$adapted(AdminManager.scala:393)}}{{at
>  
> scala.collection.TraversableLike.$anonfun$filterImpl$1(TraversableLike.scala:248)}}{{at
>  scala.collection.Iterator.foreach(Iterator.scala:929)}}{{at 
> scala.collection.Iterator.foreach$(Iterator.scala:929)}}{{at 
> scala.collection.AbstractIterator.foreach(Iterator.scala:1417)}}{{at 
> scala.collection.IterableLike.foreach(IterableLike.scala:71)}}{{at 
> scala.collection.IterableLike.foreach$(IterableLike.scala:70)}}{{at 
> scala.collection.AbstractIterable.foreach(Iterable.scala:54)}}{{at 
> scala.collection.TraversableLike.filterImpl(TraversableLike.scala:247)}}{{at 
> scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:245)}}{{at 
> scala.collection.AbstractTraversable.filterImpl(Traversable.scala:104)}}{{at 
> scala.collection.TraversableLike.filter(TraversableLike.scala:259)}}{{at 
> scala.collection.TraversableLike.filter$(TraversableLike.scala:259)}}{{at 
> scala.collection.AbstractTraversable.filter(Traversable.scala:104)}}{{at 
> kafka.server.AdminManager.createResponseConfig$1(AdminManager.scala:393)}}{{at
>  
> kafka.server.AdminManager.$anonfun$describeConfigs$1(AdminManager.scala:412)}}{{at
>  scala.collection.immutable.List.map(List.scala:283)}}{{at 
> kafka.server.AdminManager.describeConfigs(AdminManager.scala:386)}}{{at 
> kafka.server.KafkaApis.handleDescribeConfigsRequest(KafkaApis.scala:2595)}}{{at
>  kafka.server.KafkaApis.handle(KafkaApis.scala:165)}}{{at 
> kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:70)}}{{at 
> java.lang.Thread.run(Thread.java:748)}}



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

Reply via email to