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

Admin client throws UnsupportedVersion exception when talking to old broker

    XMLWordPrintableJSON

Details

    • Task
    • Status: In Progress
    • Critical
    • Resolution: Unresolved
    • 2.3.1, 2.5.0, 2.4.1
    • None
    • clients
    • None

    Description

      Users are getting this error when using a client version 2.5 against a 1.1.0 cluster/broker.

      [2020-04-28 01:09:10,663] ERROR Failed to start KSQL (io.confluent.ksql.rest.server.KsqlServerMain:63)
      
      io.confluent.ksql.util.KsqlServerException: Could not get Kafka authorized operations!
      
              at io.confluent.ksql.services.KafkaClusterUtil.isAuthorizedOperationsSupported(KafkaClusterUtil.java:51)
      
              at io.confluent.ksql.security.KsqlAuthorizationValidatorFactory.create(KsqlAuthorizationValidatorFactory.java:52)
      
              at io.confluent.ksql.rest.server.KsqlRestApplication.buildApplication(KsqlRestApplication.java:639)
      
              at io.confluent.ksql.rest.server.KsqlRestApplication.buildApplication(KsqlRestApplication.java:567)
      
              at io.confluent.ksql.rest.server.KsqlServerMain.createExecutable(KsqlServerMain.java:100)
      
              at io.confluent.ksql.rest.server.KsqlServerMain.main(KsqlServerMain.java:59)
      
      Caused by: java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.UnsupportedVersionException: Attempted to write a non-default includeClusterAuthorizedOperations at version 5
      
              at org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)
      
              at org.apache.kafka.common.internals.KafkaFutureImpl.access$000(KafkaFutureImpl.java:32)
      
              at org.apache.kafka.common.internals.KafkaFutureImpl$SingleWaiter.await(KafkaFutureImpl.java:89)
      
              at org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:260)
      
              at io.confluent.ksql.services.KafkaClusterUtil.isAuthorizedOperationsSupported(KafkaClusterUtil.java:49)
      
              ... 5 more
      
      Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: Attempted to write a non-default includeClusterAuthorizedOperations at version 5
      

      Looking at KIP-430, it mentions that the client is supposed to handle this case:

      1. Existing clients using older versions will not request authorized operations in Describe requests since the default is to disable this feature. This keeps older clients compatible with newer brokers.
      2. Newer clients connecting to older brokers will use the older protocol version and hence will not request authorized operations.
      3. When the AdminClient is talking to a broker which does not support KIP-430, it will fill in either null or UnsupportedVersionException for the returned ACL operations fields in objects. For example, `ConsumerGroupDescription#authorizedOperations` will be null if the broker did not supply this information. DescribeClusterResult#authorizedOperations will throw an `UnsupportedVersionException` if the broker did not supply this information.
      4. When new operations are added, newer brokers may return operations that are not known to older clients. AdminClient will ignore any bit that is set in authorized_operations that is not known to the client. The Set<AclOperation> created by the client from the bits returned by the broker will only include operations that the client client knows about.

      I assume that this deployment environment falls under case 2, we have this in the serialization code:

              if (_version >= 8) {
                  _writable.writeByte(includeClusterAuthorizedOperations ? (byte) 1 : (byte) 0);
              } else {
                  if (includeClusterAuthorizedOperations) {
                      throw new UnsupportedVersionException("Attempted to write a non-default includeClusterAuthorizedOperations at version " + _version);
                  }
              }
      

      I also looks like we blindly set the version independent of the Broker’s supported version:

                  MetadataRequest.Builder createRequest(int timeoutMs) {
                      // Since this only requests node information, it's safe to pass true for allowAutoTopicCreation (and it
                      // simplifies communication with older brokers)
                      return new MetadataRequest.Builder(new MetadataRequestData()
                          .setTopics(Collections.emptyList())
                          .setAllowAutoTopicCreation(true)
                          .setIncludeClusterAuthorizedOperations(options.includeAuthorizedOperations()));
                  }
      

      To implement 2. we need to make these properties ignorable.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              jagsancio Jose Armando Garcia Sancio
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated: