[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14343294#comment-14343294 ] Manikumar Reddy commented on KAFKA-1884: Created reviewboard https://reviews.apache.org/r/31627/diff/ against branch origin/trunk > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2.0 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > Attachments: KAFKA-1884.patch > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14314518#comment-14314518 ] Manikumar Reddy commented on KAFKA-1884: looks like we are already returning error code from server. It just that we are not logging any error info in new client logs. Will add some logs. > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2.0 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14311632#comment-14311632 ] Guozhang Wang commented on KAFKA-1884: -- [~omkreddy] I think this is a valid point, we should handle this exception better at the server side to return corresponding error code. > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14311252#comment-14311252 ] Manikumar Reddy commented on KAFKA-1884: [~guozhang] [~jkreps] KAFKA-1919 solves the retry problem. Currently we will get continuous empty metadata response for invalid topics. I was thinking, Can clients get the InvalidTopicException/Error code? or Can we add topic validation at client side itself? How non-java clients will handle it? > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14310548#comment-14310548 ] Pradeep Gollakota commented on KAFKA-1884: -- I guess that makes sense... I'll confirm. > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14310283#comment-14310283 ] Guozhang Wang commented on KAFKA-1884: -- That is a bit weird, the metadata requests should not persist... did you make a thread dump and check if the underlying sender thread is already gone after producer shut down? I guess it is because the background thread is not shut down when you hard kill the producer. > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14310241#comment-14310241 ] Pradeep Gollakota commented on KAFKA-1884: -- [~guozhang] That's what I figured at first. But the odd behavior is that the exception storm is happening on server even after the producer has been shut down (and the broker restarted). Not sure why that would be the case. > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14309671#comment-14309671 ] Guozhang Wang commented on KAFKA-1884: -- I think the behavior that producer's sender thread will keep retrying refreshing metadata for invalid topic names is expected, and it does not actually "block" the producer as it is a background thread; but the other issue is valid: it should back off between refreshing instead of bombarding the server. This issue is being addressed in KAFKA-1919. > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14308670#comment-14308670 ] Pradeep Gollakota commented on KAFKA-1884: -- What makes the behavior in #2 earlier even more odd is, I stopped the server, deleted the znodes, deleted the kafka log dir and restarted the server and the same behavior is seen. O.o > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14308539#comment-14308539 ] Pradeep Gollakota commented on KAFKA-1884: -- I'd like to work on this. Please assign to me. I've been able to reproduce the issue. I also noticed another oddity about this though. 1. The server side error above is being repeated 100's of times a second (each repeat increments the CorrelationId). This seems to indicate some type of retry logic. 2. If I kill the server, kill the client and start the server. The error continues to repeat. This seems to indicate that this request may be persisted somewhere. I have a good grasp of where to start looking for the problem, though I have no idea why the above two are occurring. > New Producer blocks forever for Invalid topic names > --- > > Key: KAFKA-1884 > URL: https://issues.apache.org/jira/browse/KAFKA-1884 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.8.2 >Reporter: Manikumar Reddy > Fix For: 0.8.3 > > > New producer blocks forever for invalid topics names > producer logs: > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,406] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50845,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,416] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50845. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50846,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,417] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50846. > DEBUG [2015-01-20 12:46:13,417] NetworkClient: maybeUpdateMetadata(): Trying > to send metadata request to node -1 > DEBUG [2015-01-20 12:46:13,418] NetworkClient: maybeUpdateMetadata(): Sending > metadata request ClientRequest(expectResponse=true, payload=null, > request=RequestSend(header={api_key=3,api_version=0,correlation_id=50847,client_id=my-producer}, > body={topics=[TOPIC=]})) to node -1 > TRACE [2015-01-20 12:46:13,418] NetworkClient: handleMetadataResponse(): > Ignoring empty metadata response with correlation id 50847. > Broker logs: > [2015-01-20 12:46:14,074] ERROR [KafkaApi-0] error when handling request > Name: TopicMetadataRequest; Version: 0; CorrelationId: 51020; ClientId: > my-producer; Topics: TOPIC= (kafka.server.KafkaApis) > kafka.common.InvalidTopicException: topic name TOPIC= is illegal, contains a > character other than ASCII alphanumerics, '.', '_' and '-' > at kafka.common.Topic$.validate(Topic.scala:42) > at > kafka.admin.AdminUtils$.createOrUpdateTopicPartitionAssignmentPathInZK(AdminUtils.scala:186) > at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:177) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:367) > at kafka.server.KafkaApis$$anonfun$5.apply(KafkaApis.scala:350) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > at scala.collection.immutable.Set$Set1.foreach(Set.scala:74) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > at > scala.collection.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:47) > at scala.collection.SetLike$class.map(SetLike.scala:93) > at scala.collection.AbstractSet.map(Set.scala:47) > at kafka.server.KafkaApis.getTopicMetadata(KafkaApis.scala:350) > at > kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:389) > at kafka.server.KafkaApis.handle(KafkaApis.scala:57) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) > at java.lang.Thread.run(Thread.java:722) -- This message was sent by Atlassian JIRA (v6.3.4#6332)