Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-4419

Unable to GetOffset when the ACL of topic is defined

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Duplicate
    • 0.9.0.1
    • None
    • admin, security
    • kafka 0.9.0.1
      centos 7
      kafka server with kerberos (zokeeper also with kerberos)
      listeners=PLAINTEXT://:9092,SASL_PLAINTEXT://:9093
    • Important

    Description

      I have a kafka server with kerberos enable (listeners=PLAINTEXT://:9092,SASL_PLAINTEXT://:9093), I create a test topic and I pushed some data to in. I run the GetOffsetShell to get topic offset :

      [root@kafka1 ~]# /opt/kafka/bin/kafka-run-class.sh kafka.tools.GetOffsetShell --broker-list kafka1:9092 --topic test-topic --time -1 [2016-11-17 16:52:02,471] INFO Verifying properties (kafka.utils.VerifiableProperties)
      [2016-11-17 16:52:02,479] INFO Property client.id is overridden to GetOffsetShell (kafka.utils.VerifiableProperties)
      [2016-11-17 16:52:02,479] INFO Property metadata.broker.list is overridden to kafka1:9092 (kafka.utils.VerifiableProperties)
      [2016-11-17 16:52:02,480] INFO Property request.timeout.ms is overridden to 1000 (kafka.utils.VerifiableProperties)
      [2016-11-17 16:52:02,513] INFO Fetching metadata from broker BrokerEndPoint(0,kafka1,9092) with correlation id 0 for 1 topic(s) Set(test-topic) (kafka.client.ClientUtils$)
      [2016-11-17 16:52:02,561] INFO Connected to kafka1:9092 for producing (kafka.producer.SyncProducer)
      [2016-11-17 16:52:02,573] INFO Disconnecting from kafka1:9092 (kafka.producer.SyncProducer)
      test-topic:2:773
      test-topic:1:773
      test-topic:0:772

      I added an user to ACL for my test topic:

      [root@kafka1 ~]# $KAFKA_HOME/bin/kafka-acls.sh --authorizer-properties zookeeper.connect=kafka1:2181,kafka2:2181,kafka3:2181 --add --allow-principal User:garmes --operation All --topic test-topic

      I pushed some data again. I run the GetOffsetShell to get topic offset but this time I dont have offset :

      [root@kafka1 ~]# /opt/kafka/bin/kafka-run-class.sh kafka.tools.GetOffsetShell --broker-list kafka1:9092 --topic test-topic --time -1
      [2016-11-17 16:43:31,289] INFO Verifying properties (kafka.utils.VerifiableProperties)
      [2016-11-17 16:43:31,305] INFO Property client.id is overridden to GetOffsetShell (kafka.utils.VerifiableProperties)
      [2016-11-17 16:43:31,305] INFO Property metadata.broker.list is overridden to kafka1:9092 (kafka.utils.VerifiableProperties)
      [2016-11-17 16:43:31,305] INFO Property request.timeout.ms is overridden to 1000 (kafka.utils.VerifiableProperties)
      [2016-11-17 16:43:31,339] INFO Fetching metadata from broker BrokerEndPoint(0,kafka1,9092) with correlation id 0 for 1 topic(s) Set(test-topic) (kafka.client.ClientUtils$)
      [2016-11-17 16:43:31,382] INFO Connected to kafka1:9092 for producing (kafka.producer.SyncProducer)
      [2016-11-17 16:43:31,394] INFO Disconnecting from kafka1:9092 (kafka.producer.SyncProducer)
      [root@kafka1 ~]#

      I changed the broker port from 9092 to 9093, but I have below error :

      [root@kafka1 ~]# /opt/kafka/bin/kafka-run-class.sh kafka.tools.GetOffsetShell --broker-list kafka1:9093 --topic test-topic --time -1 [2016-11-17 16:59:18,112] INFO Verifying properties (kafka.utils.VerifiableProperties)
      [2016-11-17 16:59:18,129] INFO Property client.id is overridden to GetOffsetShell (kafka.utils.VerifiableProperties)
      [2016-11-17 16:59:18,129] INFO Property metadata.broker.list is overridden to kafka1:9093 (kafka.utils.VerifiableProperties)
      [2016-11-17 16:59:18,129] INFO Property request.timeout.ms is overridden to 1000 (kafka.utils.VerifiableProperties)
      [2016-11-17 16:59:18,162] INFO Fetching metadata from broker BrokerEndPoint(0,kafka1,9093) with correlation id 0 for 1 topic(s) Set(test-topic) (kafka.client.ClientUtils$)
      [2016-11-17 16:59:18,206] INFO Connected to kafka1:9093 for producing (kafka.producer.SyncProducer)
      [2016-11-17 16:59:18,210] INFO Disconnecting from kafka1:9093 (kafka.producer.SyncProducer)
      [2016-11-17 16:59:18,212] WARN Fetching topic metadata with correlation id 0 for topics [Set(test-topic)] from broker [BrokerEndPoint(0,kafka1,9093)] failed (kafka.client.ClientUtils$)
      java.io.EOFException
      at org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:83)
      at kafka.network.BlockingChannel.readCompletely(BlockingChannel.scala:129)
      at kafka.network.BlockingChannel.receive(BlockingChannel.scala:120)
      at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:77)
      at kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:74)
      at kafka.producer.SyncProducer.send(SyncProducer.scala:119)
      at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:59)
      at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:94)
      at kafka.tools.GetOffsetShell$.main(GetOffsetShell.scala:78)
      at kafka.tools.GetOffsetShell.main(GetOffsetShell.scala)
      [2016-11-17 16:59:18,213] INFO Disconnecting from kafka1:9093 (kafka.producer.SyncProducer)
      Exception in thread "main" kafka.common.KafkaException: fetching topic metadata for topics [Set(test-topic)] from broker [ArrayBuffer(BrokerEndPoint(0,kafka1,9093))] failed
      at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:73)
      at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:94)
      at kafka.tools.GetOffsetShell$.main(GetOffsetShell.scala:78)
      at kafka.tools.GetOffsetShell.main(GetOffsetShell.scala)
      Caused by: java.io.EOFException
      at org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:83)
      at kafka.network.BlockingChannel.readCompletely(BlockingChannel.scala:129)
      at kafka.network.BlockingChannel.receive(BlockingChannel.scala:120)
      at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:77)
      at kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:74)
      at kafka.producer.SyncProducer.send(SyncProducer.scala:119)
      at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:59)
      ... 3 more
      [root@kafka1 ~]#

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              magarmes Mohammed amine GARMES
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - 12h
                  12h
                  Remaining:
                  Remaining Estimate - 12h
                  12h
                  Logged:
                  Time Spent - Not Specified
                  Not Specified