class ClientCompatibilityFeaturesTest(Test):
    """
    Tests clients for the presence or absence of specific features when communicating with brokers with various
    versions. Relies on ClientCompatibilityTest.java for much of the functionality.
    """
    def __init__(self, test_context):
        """:type test_context: ducktape.tests.test.TestContext"""
        super(ClientCompatibilityFeaturesTest,
              self).__init__(test_context=test_context)

        self.zk = ZookeeperService(test_context, num_nodes=3)

        # Generate a unique topic name
        topic_name = "client_compat_features_topic_%d%d" % (int(
            time.time()), randint(0, 2147483647))
        self.topics = {
            topic_name: {
                "partitions":
                1,  # Use only one partition to avoid worrying about ordering
                "replication-factor": 3
            }
        }
        self.kafka = KafkaService(test_context,
                                  num_nodes=3,
                                  zk=self.zk,
                                  topics=self.topics)

    def invoke_compatibility_program(self, features):
        # Run the compatibility test on the first Kafka node.
        node = self.zk.nodes[0]
        cmd = ("%s org.apache.kafka.tools.ClientCompatibilityTest "
               "--bootstrap-server %s "
               "--num-cluster-nodes %d "
               "--topic %s " % (self.zk.path.script(
                   "kafka-run-class.sh", node), self.kafka.bootstrap_servers(),
                                len(self.kafka.nodes), self.topics.keys()[0]))
        for k, v in features.iteritems():
            cmd = cmd + ("--%s %s " % (k, v))
        results_dir = TestContext.results_dir(self.test_context, 0)
        os.makedirs(results_dir)
        ssh_log_file = "%s/%s" % (results_dir,
                                  "client_compatibility_test_output.txt")
        try:
            self.logger.info("Running %s" % cmd)
            run_command(node, cmd, ssh_log_file)
        except Exception as e:
            self.logger.info("** Command failed.  See %s for log messages." %
                             ssh_log_file)
            raise

    @parametrize(broker_version=str(DEV_BRANCH))
    @parametrize(broker_version=str(LATEST_0_10_0))
    @parametrize(broker_version=str(LATEST_0_10_1))
    @parametrize(broker_version=str(LATEST_0_10_2))
    def run_compatibility_test(self, broker_version):
        self.zk.start()
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()
        features = get_broker_features(broker_version)
        self.invoke_compatibility_program(features)
class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest):
    """
    These tests validate that we can use a new client to produce and consume from older brokers.
    """

    def __init__(self, test_context):
        """:type test_context: ducktape.tests.test.TestContext"""
        super(ClientCompatibilityProduceConsumeTest, self).__init__(test_context=test_context)

        self.topic = "test_topic"
        self.zk = ZookeeperService(test_context, num_nodes=3)
        self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics={self.topic:{
                                                                    "partitions": 10,
                                                                    "replication-factor": 2}})
        self.num_partitions = 10
        self.timeout_sec = 60
        self.producer_throughput = 1000
        self.num_producers = 2
        self.messages_per_producer = 1000
        self.num_consumers = 1

    def setUp(self):
        self.zk.start()

    def min_cluster_size(self):
        # Override this since we're adding services outside of the constructor
        return super(ClientCompatibilityProduceConsumeTest, self).min_cluster_size() + self.num_producers + self.num_consumers

    @parametrize(broker_version=str(DEV_BRANCH))
    @parametrize(broker_version=str(LATEST_0_10_0))
    @parametrize(broker_version=str(LATEST_0_10_1))
    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_11_0))
    @parametrize(broker_version=str(LATEST_1_0))
    @parametrize(broker_version=str(LATEST_1_1))
    @parametrize(broker_version=str(LATEST_2_0))
    @parametrize(broker_version=str(LATEST_2_1))
    @parametrize(broker_version=str(LATEST_2_2))
    @parametrize(broker_version=str(LATEST_2_3))
    @parametrize(broker_version=str(LATEST_2_4))
    @parametrize(broker_version=str(LATEST_2_5))
    def test_produce_consume(self, broker_version):
        print("running producer_consumer_compat with broker_version = %s" % broker_version, flush=True)
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.security_protocol = "PLAINTEXT"
        self.kafka.interbroker_security_protocol = self.kafka.security_protocol
        self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka,
                                           self.topic, throughput=self.producer_throughput,
                                           message_validator=is_int_with_prefix)
        self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic,
                                        consumer_timeout_ms=60000,
                                        message_validator=is_int_with_prefix)
        self.kafka.start()

        self.run_produce_consume_validate(lambda: wait_until(
            lambda: self.producer.each_produced_at_least(self.messages_per_producer) == True,
            timeout_sec=120, backoff_sec=1,
            err_msg="Producer did not produce all messages in reasonable amount of time"))
class ClientCompatibilityFeaturesTest(Test):
    """
    Tests clients for the presence or absence of specific features when communicating with brokers with various
    versions. Relies on ClientCompatibilityTest.java for much of the functionality.
    """

    def __init__(self, test_context):
        """:type test_context: ducktape.tests.test.TestContext"""
        super(ClientCompatibilityFeaturesTest, self).__init__(test_context=test_context)

        self.zk = ZookeeperService(test_context, num_nodes=3)

        # Generate a unique topic name
        topic_name = "client_compat_features_topic_%d%d" % (int(time.time()), randint(0, 2147483647))
        self.topics = { topic_name: {
            "partitions": 1, # Use only one partition to avoid worrying about ordering
            "replication-factor": 3
            }}
        self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics=self.topics)

    def invoke_compatibility_program(self, features):
        # Run the compatibility test on the first Kafka node.
        node = self.zk.nodes[0]
        cmd = ("%s org.apache.kafka.tools.ClientCompatibilityTest "
               "--bootstrap-server %s "
               "--num-cluster-nodes %d "
               "--topic %s " % (self.zk.path.script("kafka-run-class.sh", node),
                               self.kafka.bootstrap_servers(),
                               len(self.kafka.nodes),
                               self.topics.keys()[0]))
        for k, v in features.iteritems():
            cmd = cmd + ("--%s %s " % (k, v))
        results_dir = TestContext.results_dir(self.test_context, 0)
        os.makedirs(results_dir)
        ssh_log_file = "%s/%s" % (results_dir, "client_compatibility_test_output.txt")
        try:
          self.logger.info("Running %s" % cmd)
          run_command(node, cmd, ssh_log_file)
        except Exception as e:
          self.logger.info("** Command failed.  See %s for log messages." % ssh_log_file)
          raise

    @parametrize(broker_version=str(DEV_BRANCH))
    @parametrize(broker_version=str(LATEST_0_10_0))
    @parametrize(broker_version=str(LATEST_0_10_1))
    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_11_0))
    @parametrize(broker_version=str(LATEST_1_0))
    @parametrize(broker_version=str(LATEST_1_1))
    def run_compatibility_test(self, broker_version):
        self.zk.start()
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()
        features = get_broker_features(broker_version)
        self.invoke_compatibility_program(features)
class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest):
    """
    These tests validate that we can use a new client to produce and consume from older brokers.
    """

    def __init__(self, test_context):
        """:type test_context: ducktape.tests.test.TestContext"""
        super(ClientCompatibilityProduceConsumeTest, self).__init__(test_context=test_context)

        self.topic = "test_topic"
        self.zk = ZookeeperService(test_context, num_nodes=3)
        self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics={self.topic:{
                                                                    "partitions": 10,
                                                                    "replication-factor": 2}})
        self.num_partitions = 10
        self.timeout_sec = 60
        self.producer_throughput = 1000
        self.num_producers = 2
        self.messages_per_producer = 1000
        self.num_consumers = 1

    def setUp(self):
        self.zk.start()

    def min_cluster_size(self):
        # Override this since we're adding services outside of the constructor
        return super(ClientCompatibilityProduceConsumeTest, self).min_cluster_size() + self.num_producers + self.num_consumers

    @parametrize(broker_version=str(DEV_BRANCH))
    @parametrize(broker_version=str(LATEST_0_10_0))
    @parametrize(broker_version=str(LATEST_0_10_1))
    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_11_0))
    @parametrize(broker_version=str(LATEST_1_0))
    @parametrize(broker_version=str(LATEST_1_1))
    def test_produce_consume(self, broker_version):
        print("running producer_consumer_compat with broker_version = %s" % broker_version)
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.security_protocol = "PLAINTEXT"
        self.kafka.interbroker_security_protocol = self.kafka.security_protocol
        self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka,
                                           self.topic, throughput=self.producer_throughput,
                                           message_validator=is_int_with_prefix)
        self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic,
                                        consumer_timeout_ms=60000,
                                        message_validator=is_int_with_prefix)
        self.kafka.start()

        self.run_produce_consume_validate(lambda: wait_until(
            lambda: self.producer.each_produced_at_least(self.messages_per_producer) == True,
            timeout_sec=120, backoff_sec=1,
            err_msg="Producer did not produce all messages in reasonable amount of time"))
Esempio n. 5
0
class StreamsBrokerCompatibility(Test):
    """
    These tests validates that
    - Streams 0.11+ w/ EOS fails fast for older brokers 0.10.2 and 0.10.1
    - Streams 0.11+ w/o EOS works for older brokers 0.10.2 and 0.10.1
    - Streams fails fast for 0.10.0 brokers
    - Streams times-out for pre-0.10.0 brokers
    """

    input = "brokerCompatibilitySourceTopic"
    output = "brokerCompatibilitySinkTopic"

    def __init__(self, test_context):
        super(StreamsBrokerCompatibility,
              self).__init__(test_context=test_context)
        self.zk = ZookeeperService(test_context, num_nodes=1)
        self.kafka = KafkaService(test_context,
                                  num_nodes=1,
                                  zk=self.zk,
                                  topics={
                                      self.input: {
                                          'partitions': 1,
                                          'replication-factor': 1
                                      },
                                      self.output: {
                                          'partitions': 1,
                                          'replication-factor': 1
                                      }
                                  })
        self.consumer = VerifiableConsumer(
            test_context, 1, self.kafka, self.output,
            "stream-broker-compatibility-verify-consumer")

    def setUp(self):
        self.zk.start()

    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_10_1))
    def test_fail_fast_on_incompatible_brokers_if_eos_enabled(
            self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka, True)
        processor.start()

        processor.node.account.ssh(processor.start_cmd(processor.node))
        with processor.node.account.monitor_log(
                processor.STDERR_FILE) as monitor:
            monitor.wait_until(
                'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS ',
                timeout_sec=60,
                err_msg=
                "Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS ' error message "
                + str(processor.node.account))

        self.kafka.stop()

    @parametrize(broker_version=str(LATEST_0_11_0))
    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_10_1))
    def test_compatible_brokers_eos_disabled(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka, False)
        processor.start()

        self.consumer.start()

        processor.wait()

        wait_until(
            lambda: self.consumer.total_consumed() > 0,
            timeout_sec=30,
            err_msg=
            "Did expect to read a message but got none within 30 seconds.")

        self.consumer.stop()
        self.kafka.stop()

    @parametrize(broker_version=str(LATEST_0_10_0))
    def test_fail_fast_on_incompatible_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka, False)
        processor.start()

        processor.node.account.ssh(processor.start_cmd(processor.node))
        with processor.node.account.monitor_log(
                processor.STDERR_FILE) as monitor:
            monitor.wait_until(
                'FATAL: An unexpected exception org.apache.kafka.streams.errors.StreamsException: Could not create internal topics.',
                timeout_sec=60,
                err_msg=
                "Never saw 'FATAL: An unexpected exception org.apache.kafka.streams.errors.StreamsException: Could not create internal topics.' error message "
                + str(processor.node.account))

        self.kafka.stop()

    @ignore
    @parametrize(broker_version=str(LATEST_0_9))
    @parametrize(broker_version=str(LATEST_0_8_2))
    def test_timeout_on_pre_010_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka, False)
        processor.start()

        processor.node.account.ssh(processor.start_cmd(processor.node))
        with processor.node.account.monitor_log(
                processor.STDERR_FILE) as monitor:
            monitor.wait_until(
                'Exception in thread "main" org.apache.kafka.streams.errors.BrokerNotFoundException: Could not find any available broker.',
                timeout_sec=60,
                err_msg="Never saw 'no available brokers' error message " +
                str(processor.node.account))

        self.kafka.stop()
Esempio n. 6
0
class ClientCompatibilityFeaturesTest(Test):
    """
    Tests clients for the presence or absence of specific features when communicating with brokers with various
    versions. Relies on ClientCompatibilityTest.java for much of the functionality.
    """
    def __init__(self, test_context):
        """:type test_context: ducktape.tests.test.TestContext"""
        super(ClientCompatibilityFeaturesTest,
              self).__init__(test_context=test_context)

        self.zk = ZookeeperService(test_context,
                                   num_nodes=3) if quorum.for_test(
                                       test_context) == quorum.zk else None

        # Generate a unique topic name
        topic_name = "client_compat_features_topic_%d%d" % (int(
            time.time()), randint(0, 2147483647))
        self.topics = {
            topic_name: {
                "partitions":
                1,  # Use only one partition to avoid worrying about ordering
                "replication-factor": 3
            }
        }
        self.kafka = KafkaService(test_context,
                                  num_nodes=3,
                                  zk=self.zk,
                                  topics=self.topics)
        # Always use the latest version of org.apache.kafka.tools.ClientCompatibilityTest
        # so store away the path to the DEV version before we set the Kafka version
        self.dev_script_path = self.kafka.path.script("kafka-run-class.sh",
                                                      self.kafka.nodes[0])

    def invoke_compatibility_program(self, features):
        # Run the compatibility test on the first Kafka node.
        node = self.kafka.nodes[0]
        cmd = ("%s org.apache.kafka.tools.ClientCompatibilityTest "
               "--bootstrap-server %s "
               "--num-cluster-nodes %d "
               "--topic %s " %
               (self.dev_script_path, self.kafka.bootstrap_servers(),
                len(self.kafka.nodes), list(self.topics.keys())[0]))
        for k, v in features.items():
            cmd = cmd + ("--%s %s " % (k, v))
        results_dir = TestContext.results_dir(self.test_context, 0)
        try:
            os.makedirs(results_dir)
        except OSError as e:
            if e.errno == errno.EEXIST and os.path.isdir(results_dir):
                pass
            else:
                raise
        ssh_log_file = "%s/%s" % (results_dir,
                                  "client_compatibility_test_output.txt")
        try:
            self.logger.info("Running %s" % cmd)
            run_command(node, cmd, ssh_log_file)
        except Exception as e:
            self.logger.info("** Command failed.  See %s for log messages." %
                             ssh_log_file)
            raise

    @cluster(num_nodes=7)
    @matrix(broker_version=[str(DEV_BRANCH)],
            metadata_quorum=quorum.all_non_upgrade)
    @parametrize(broker_version=str(LATEST_0_10_0))
    @parametrize(broker_version=str(LATEST_0_10_1))
    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_11_0))
    @parametrize(broker_version=str(LATEST_1_0))
    @parametrize(broker_version=str(LATEST_1_1))
    @parametrize(broker_version=str(LATEST_2_0))
    @parametrize(broker_version=str(LATEST_2_1))
    @parametrize(broker_version=str(LATEST_2_2))
    @parametrize(broker_version=str(LATEST_2_3))
    @parametrize(broker_version=str(LATEST_2_4))
    @parametrize(broker_version=str(LATEST_2_5))
    @parametrize(broker_version=str(LATEST_2_6))
    @parametrize(broker_version=str(LATEST_2_7))
    @parametrize(broker_version=str(LATEST_2_8))
    @parametrize(broker_version=str(LATEST_3_0))
    @parametrize(broker_version=str(LATEST_3_1))
    def run_compatibility_test(self,
                               broker_version,
                               metadata_quorum=quorum.zk):
        if self.zk:
            self.zk.start()
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()
        features = get_broker_features(broker_version)
        self.invoke_compatibility_program(features)
Esempio n. 7
0
class StreamsBrokerCompatibility(Test):
    """
    These tests validate that Streams v0.10.2+ can connect to older brokers v0.10.1+
    and that Streams fails fast for pre-0.10.0 brokers
    """

    input = "brokerCompatibilitySourceTopic"
    output = "brokerCompatibilitySinkTopic"

    def __init__(self, test_context):
        super(StreamsBrokerCompatibility,
              self).__init__(test_context=test_context)

        self.zk = ZookeeperService(test_context, num_nodes=1)
        self.kafka = KafkaService(test_context,
                                  num_nodes=1,
                                  zk=self.zk,
                                  topics={
                                      self.input: {
                                          'partitions': 1,
                                          'replication-factor': 1
                                      },
                                      self.output: {
                                          'partitions': 1,
                                          'replication-factor': 1
                                      }
                                  })

        self.processor = StreamsBrokerCompatibilityService(
            self.test_context, self.kafka)

        self.consumer = VerifiableConsumer(
            test_context, 1, self.kafka, self.output,
            "stream-broker-compatibility-verify-consumer")

    def setUp(self):
        self.zk.start()

    @parametrize(broker_version=str(DEV_BRANCH))
    @parametrize(broker_version=str(LATEST_0_10_1))
    def test_compatible_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        self.processor.start()
        self.consumer.start()

        self.processor.wait()

        wait_until(
            lambda: self.consumer.total_consumed() > 0,
            timeout_sec=30,
            err_msg=
            "Did expect to read a message but got none within 30 seconds.")

        self.consumer.stop()
        self.kafka.stop()

    @parametrize(broker_version=str(LATEST_0_10_0))
    def test_fail_fast_on_incompatible_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        self.processor.start()

        self.processor.node.account.ssh(
            self.processor.start_cmd(self.processor.node))
        with self.processor.node.account.monitor_log(
                self.processor.STDERR_FILE) as monitor:
            monitor.wait_until(
                'Exception in thread "main" org.apache.kafka.streams.errors.StreamsException: Kafka Streams requires broker version 0.10.1.x or higher.',
                timeout_sec=60,
                err_msg="Never saw 'incompatible broker' error message " +
                str(self.processor.node.account))

        self.kafka.stop()
class StreamsBrokerCompatibility(Test):
    """
    These tests validates that
    - Streams works for older brokers 0.11 (or newer)
    - Streams w/ EOS-alpha works for older brokers 0.11 (or newer)
    - Streams w/ EOS-beta works for older brokers 2.5 (or newer)
    - Streams fails fast for older brokers 0.10.0, 0.10.2, and 0.10.1
    - Streams w/ EOS-beta fails fast for older brokers 2.4 or older
    """

    input = "brokerCompatibilitySourceTopic"
    output = "brokerCompatibilitySinkTopic"

    def __init__(self, test_context):
        super(StreamsBrokerCompatibility,
              self).__init__(test_context=test_context)
        self.zk = ZookeeperService(test_context, num_nodes=1)
        self.kafka = KafkaService(
            test_context,
            num_nodes=1,
            zk=self.zk,
            topics={
                self.input: {
                    'partitions': 1,
                    'replication-factor': 1
                },
                self.output: {
                    'partitions': 1,
                    'replication-factor': 1
                }
            },
            server_prop_overides=[[
                "transaction.state.log.replication.factor", "1"
            ], ["transaction.state.log.min.isr", "1"]])
        self.consumer = VerifiableConsumer(
            test_context, 1, self.kafka, self.output,
            "stream-broker-compatibility-verify-consumer")

    def setUp(self):
        self.zk.start()

    @parametrize(broker_version=str(LATEST_2_4))
    @parametrize(broker_version=str(LATEST_2_3))
    @parametrize(broker_version=str(LATEST_2_2))
    @parametrize(broker_version=str(LATEST_2_1))
    @parametrize(broker_version=str(LATEST_2_0))
    @parametrize(broker_version=str(LATEST_1_1))
    @parametrize(broker_version=str(LATEST_1_0))
    @parametrize(broker_version=str(LATEST_0_11_0))
    def test_compatible_brokers_eos_disabled(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka,
                                                      "at_least_once")
        processor.start()

        self.consumer.start()

        processor.wait()

        wait_until(
            lambda: self.consumer.total_consumed() > 0,
            timeout_sec=30,
            err_msg=
            "Did expect to read a message but got none within 30 seconds.")

        self.consumer.stop()
        self.kafka.stop()

    @parametrize(broker_version=str(LATEST_2_5))
    @parametrize(broker_version=str(LATEST_2_4))
    @parametrize(broker_version=str(LATEST_2_3))
    @parametrize(broker_version=str(LATEST_2_2))
    @parametrize(broker_version=str(LATEST_2_1))
    @parametrize(broker_version=str(LATEST_2_0))
    @parametrize(broker_version=str(LATEST_1_1))
    @parametrize(broker_version=str(LATEST_1_0))
    @parametrize(broker_version=str(LATEST_0_11_0))
    def test_compatible_brokers_eos_alpha_enabled(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka,
                                                      "exactly_once")
        processor.start()

        self.consumer.start()

        processor.wait()

        wait_until(
            lambda: self.consumer.total_consumed() > 0,
            timeout_sec=30,
            err_msg=
            "Did expect to read a message but got none within 30 seconds.")

        self.consumer.stop()
        self.kafka.stop()

    # TODO enable after 2.5 is released
    # @parametrize(broker_version=str(LATEST_2_5))
    # def test_compatible_brokers_eos_beta_enabled(self, broker_version):
    #     self.kafka.set_version(KafkaVersion(broker_version))
    #     self.kafka.start()
    #
    #     processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, "exactly_once_beta")
    #     processor.start()
    #
    #     self.consumer.start()
    #
    #     processor.wait()
    #
    #     wait_until(lambda: self.consumer.total_consumed() > 0, timeout_sec=30, err_msg="Did expect to read a message but got none within 30 seconds.")
    #
    #     self.consumer.stop()
    #     self.kafka.stop()

    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_10_1))
    @parametrize(broker_version=str(LATEST_0_10_0))
    def test_fail_fast_on_incompatible_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka,
                                                      "at_least_once")

        with processor.node.account.monitor_log(
                processor.STDERR_FILE) as monitor:
            processor.start()
            monitor.wait_until(
                'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException',
                timeout_sec=60,
                err_msg=
                "Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException "
                + str(processor.node.account))

        self.kafka.stop()

    @parametrize(broker_version=str(LATEST_2_4))
    @parametrize(broker_version=str(LATEST_2_3))
    @parametrize(broker_version=str(LATEST_2_2))
    @parametrize(broker_version=str(LATEST_2_1))
    @parametrize(broker_version=str(LATEST_2_0))
    @parametrize(broker_version=str(LATEST_1_1))
    @parametrize(broker_version=str(LATEST_1_0))
    @parametrize(broker_version=str(LATEST_0_11_0))
    def test_fail_fast_on_incompatible_brokers_if_eos_beta_enabled(
            self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context,
                                                      self.kafka,
                                                      "exactly_once_beta")

        with processor.node.account.monitor_log(
                processor.STDERR_FILE) as monitor:
            with processor.node.account.monitor_log(processor.LOG_FILE) as log:
                processor.start()
                log.wait_until(
                    'Shutting down because the Kafka cluster seems to be on a too old version. Setting processing\.guarantee="exactly_once_beta" requires broker version 2\.5 or higher\.',
                    timeout_sec=60,
                    err_msg=
                    "Never saw 'Shutting down, because the Kafka cluster seems to be on a too old version. Setting `processing.guarantee=\"exaclty_once_beta\"` requires broker version 2.5 or higher.' log message "
                    + str(processor.node.account))
                monitor.wait_until(
                    'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException',
                    timeout_sec=60,
                    err_msg=
                    "Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException' error message "
                    + str(processor.node.account))

        self.kafka.stop()
class StreamsBrokerCompatibility(Test):
    """
    These tests validate that Streams v0.10.2+ can connect to older brokers v0.10+
    and that Streams fails fast for pre-0.10 brokers
    """

    input = "brokerCompatibilitySourceTopic"
    output = "brokerCompatibilitySinkTopic"

    def __init__(self, test_context):
        super(StreamsBrokerCompatibility, self).__init__(test_context=test_context)

        self.zk = ZookeeperService(test_context, num_nodes=1)
        self.kafka = KafkaService(test_context,
                                  num_nodes=1,
                                  zk=self.zk,
                                  topics={
                                      self.input: {'partitions': 1, 'replication-factor': 1},
                                      self.output: {'partitions': 1, 'replication-factor': 1}
                                  })

        self.processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka)

        self.consumer = VerifiableConsumer(test_context,
                                           1,
                                           self.kafka,
                                           self.output,
                                           "stream-broker-compatibility-verify-consumer")

    def setUp(self):
        self.zk.start()

    @parametrize(broker_version=str(DEV_BRANCH))
    @parametrize(broker_version=str(LATEST_0_10_1))
    def test_compatible_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        self.processor.start()
        self.consumer.start()

        self.processor.wait()

        num_consumed_mgs = self.consumer.total_consumed()

        self.consumer.stop()
        self.kafka.stop()

        assert num_consumed_mgs == 1, \
            "Did expect to read exactly one message but got %d" % num_consumed_mgs

    @parametrize(broker_version=str(LATEST_0_10_0))
    def test_fail_fast_on_incompatible_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        self.processor.start()

        self.processor.node.account.ssh(self.processor.start_cmd(self.processor.node))
        with self.processor.node.account.monitor_log(self.processor.STDERR_FILE) as monitor:
            monitor.wait_until('Exception in thread "main" org.apache.kafka.streams.errors.StreamsException: Kafka Streams requires broker version 0.10.1.x or higher.',
                        timeout_sec=60,
                        err_msg="Never saw 'incompatible broker' error message " + str(self.processor.node.account))

        self.kafka.stop()
Esempio n. 10
0
class QuotaTest(Test):
    """
    These tests verify that quota provides expected functionality -- they run
    producer, broker, and consumer with different clientId and quota configuration and
    check that the observed throughput is close to the value we expect.
    """
    def __init__(self, test_context):
        """:type test_context: ducktape.tests.test.TestContext"""
        super(QuotaTest, self).__init__(test_context=test_context)

        self.topic = 'test_topic'
        self.logger.info('use topic ' + self.topic)

        self.maximum_client_deviation_percentage = 100.0
        self.maximum_broker_deviation_percentage = 5.0
        self.num_records = 50000
        self.record_size = 3000

        self.zk = ZookeeperService(test_context, num_nodes=1)
        self.kafka = KafkaService(
            test_context,
            num_nodes=1,
            zk=self.zk,
            security_protocol='SSL',
            authorizer_class_name='',
            interbroker_security_protocol='SSL',
            topics={
                self.topic: {
                    'partitions': 6,
                    'replication-factor': 1,
                    'configs': {
                        'min.insync.replicas': 1
                    }
                }
            },
            jmx_object_names=[
                'kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec',
                'kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec'
            ],
            jmx_attributes=['OneMinuteRate'])
        self.num_producers = 1
        self.num_consumers = 2

    def setUp(self):
        self.zk.start()

    def min_cluster_size(self):
        """Override this since we're adding services outside of the constructor"""
        return super(
            QuotaTest,
            self).min_cluster_size() + self.num_producers + self.num_consumers

    @cluster(num_nodes=5)
    @matrix(quota_type=[
        QuotaConfig.CLIENT_ID, QuotaConfig.USER, QuotaConfig.USER_CLIENT
    ],
            override_quota=[True, False])
    @parametrize(quota_type=QuotaConfig.CLIENT_ID, consumer_num=2)
    @parametrize(quota_type=QuotaConfig.CLIENT_ID,
                 old_broker_throttling_behavior=True)
    @parametrize(quota_type=QuotaConfig.CLIENT_ID,
                 old_client_throttling_behavior=True)
    def test_quota(self,
                   quota_type,
                   override_quota=True,
                   producer_num=1,
                   consumer_num=1,
                   old_broker_throttling_behavior=False,
                   old_client_throttling_behavior=False):
        # Old (pre-2.0) throttling behavior for broker throttles before sending a response to the client.
        if old_broker_throttling_behavior:
            self.kafka.set_version(LATEST_1_1)
        self.kafka.start()

        self.quota_config = QuotaConfig(quota_type, override_quota, self.kafka)
        producer_client_id = self.quota_config.client_id
        consumer_client_id = self.quota_config.client_id

        # Old (pre-2.0) throttling behavior for client does not throttle upon receiving a response with a non-zero throttle time.
        if old_client_throttling_behavior:
            client_version = LATEST_1_1
        else:
            client_version = DEV_BRANCH

        # Produce all messages
        producer = ProducerPerformanceService(self.test_context,
                                              producer_num,
                                              self.kafka,
                                              topic=self.topic,
                                              num_records=self.num_records,
                                              record_size=self.record_size,
                                              throughput=-1,
                                              client_id=producer_client_id,
                                              version=client_version)

        producer.run()

        # Consume all messages
        consumer = ConsoleConsumer(
            self.test_context,
            consumer_num,
            self.kafka,
            self.topic,
            consumer_timeout_ms=60000,
            client_id=consumer_client_id,
            jmx_object_names=[
                'kafka.consumer:type=consumer-fetch-manager-metrics,client-id=%s'
                % consumer_client_id
            ],
            jmx_attributes=['bytes-consumed-rate'],
            version=client_version)
        consumer.run()

        for idx, messages in consumer.messages_consumed.iteritems():
            assert len(
                messages
            ) > 0, "consumer %d didn't consume any message before timeout" % idx

        success, msg = self.validate(self.kafka, producer, consumer)
        assert success, msg

    def validate(self, broker, producer, consumer):
        """
        For each client_id we validate that:
        1) number of consumed messages equals number of produced messages
        2) maximum_producer_throughput <= producer_quota * (1 + maximum_client_deviation_percentage/100)
        3) maximum_broker_byte_in_rate <= producer_quota * (1 + maximum_broker_deviation_percentage/100)
        4) maximum_consumer_throughput <= consumer_quota * (1 + maximum_client_deviation_percentage/100)
        5) maximum_broker_byte_out_rate <= consumer_quota * (1 + maximum_broker_deviation_percentage/100)
        """
        success = True
        msg = ''

        self.kafka.read_jmx_output_all_nodes()

        # validate that number of consumed messages equals number of produced messages
        produced_num = sum([value['records'] for value in producer.results])
        consumed_num = sum(
            [len(value) for value in consumer.messages_consumed.values()])
        self.logger.info('producer produced %d messages' % produced_num)
        self.logger.info('consumer consumed %d messages' % consumed_num)
        if produced_num != consumed_num:
            success = False
            msg += "number of produced messages %d doesn't equal number of consumed messages %d" % (
                produced_num, consumed_num)

        # validate that maximum_producer_throughput <= producer_quota * (1 + maximum_client_deviation_percentage/100)
        producer_maximum_bps = max(
            metric.value
            for k, metrics in producer.metrics(group='producer-metrics',
                                               name='outgoing-byte-rate',
                                               client_id=producer.client_id)
            for metric in metrics)
        producer_quota_bps = self.quota_config.producer_quota
        self.logger.info(
            'producer has maximum throughput %.2f bps with producer quota %.2f bps'
            % (producer_maximum_bps, producer_quota_bps))
        if producer_maximum_bps > producer_quota_bps * (
                self.maximum_client_deviation_percentage / 100 + 1):
            success = False
            msg += 'maximum producer throughput %.2f bps exceeded producer quota %.2f bps by more than %.1f%%' % \
                   (producer_maximum_bps, producer_quota_bps, self.maximum_client_deviation_percentage)

        # validate that maximum_broker_byte_in_rate <= producer_quota * (1 + maximum_broker_deviation_percentage/100)
        broker_byte_in_attribute_name = 'kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec:OneMinuteRate'
        broker_maximum_byte_in_bps = broker.maximum_jmx_value[
            broker_byte_in_attribute_name]
        self.logger.info(
            'broker has maximum byte-in rate %.2f bps with producer quota %.2f bps'
            % (broker_maximum_byte_in_bps, producer_quota_bps))
        if broker_maximum_byte_in_bps > producer_quota_bps * (
                self.maximum_broker_deviation_percentage / 100 + 1):
            success = False
            msg += 'maximum broker byte-in rate %.2f bps exceeded producer quota %.2f bps by more than %.1f%%' % \
                   (broker_maximum_byte_in_bps, producer_quota_bps, self.maximum_broker_deviation_percentage)

        # validate that maximum_consumer_throughput <= consumer_quota * (1 + maximum_client_deviation_percentage/100)
        consumer_attribute_name = 'kafka.consumer:type=consumer-fetch-manager-metrics,client-id=%s:bytes-consumed-rate' % consumer.client_id
        consumer_maximum_bps = consumer.maximum_jmx_value[
            consumer_attribute_name]
        consumer_quota_bps = self.quota_config.consumer_quota
        self.logger.info(
            'consumer has maximum throughput %.2f bps with consumer quota %.2f bps'
            % (consumer_maximum_bps, consumer_quota_bps))
        if consumer_maximum_bps > consumer_quota_bps * (
                self.maximum_client_deviation_percentage / 100 + 1):
            success = False
            msg += 'maximum consumer throughput %.2f bps exceeded consumer quota %.2f bps by more than %.1f%%' % \
                   (consumer_maximum_bps, consumer_quota_bps, self.maximum_client_deviation_percentage)

        # validate that maximum_broker_byte_out_rate <= consumer_quota * (1 + maximum_broker_deviation_percentage/100)
        broker_byte_out_attribute_name = 'kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec:OneMinuteRate'
        broker_maximum_byte_out_bps = broker.maximum_jmx_value[
            broker_byte_out_attribute_name]
        self.logger.info(
            'broker has maximum byte-out rate %.2f bps with consumer quota %.2f bps'
            % (broker_maximum_byte_out_bps, consumer_quota_bps))
        if broker_maximum_byte_out_bps > consumer_quota_bps * (
                self.maximum_broker_deviation_percentage / 100 + 1):
            success = False
            msg += 'maximum broker byte-out rate %.2f bps exceeded consumer quota %.2f bps by more than %.1f%%' % \
                   (broker_maximum_byte_out_bps, consumer_quota_bps, self.maximum_broker_deviation_percentage)

        return success, msg
class StreamsBrokerCompatibility(Test):
    """
    These tests validates that
    - Streams 0.11+ w/ EOS fails fast for older brokers 0.10.2 and 0.10.1
    - Streams 0.11+ w/o EOS works for older brokers 0.10.2 and 0.10.1
    - Streams fails fast for 0.10.0 brokers
    - Streams times-out for pre-0.10.0 brokers
    """

    input = "brokerCompatibilitySourceTopic"
    output = "brokerCompatibilitySinkTopic"

    def __init__(self, test_context):
        super(StreamsBrokerCompatibility, self).__init__(test_context=test_context)
        self.zk = ZookeeperService(test_context, num_nodes=1)
        self.kafka = KafkaService(test_context,
                                  num_nodes=1,
                                  zk=self.zk,
                                  topics={
                                      self.input: {'partitions': 1, 'replication-factor': 1},
                                      self.output: {'partitions': 1, 'replication-factor': 1}
                                  })
        self.consumer = VerifiableConsumer(test_context,
                                           1,
                                           self.kafka,
                                           self.output,
                                           "stream-broker-compatibility-verify-consumer")

    def setUp(self):
        self.zk.start()

   
    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_10_1))
    def test_fail_fast_on_incompatible_brokers_if_eos_enabled(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, True)

        with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor:
            processor.start()
            monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: Cannot create a v0 FindCoordinator request because we require features supported only in 1 or later.',
                               timeout_sec=60,
                               err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: Cannot create a v0 FindCoordinator request because we require features supported only in 1 or later.' error message " + str(processor.node.account))

        self.kafka.stop()

    @parametrize(broker_version=str(LATEST_0_11_0))
    @parametrize(broker_version=str(LATEST_0_10_2))
    @parametrize(broker_version=str(LATEST_0_10_1))
    def test_compatible_brokers_eos_disabled(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, False)
        processor.start()

        self.consumer.start()

        processor.wait()

        wait_until(lambda: self.consumer.total_consumed() > 0, timeout_sec=30, err_msg="Did expect to read a message but got none within 30 seconds.")

        self.consumer.stop()
        self.kafka.stop()

    @parametrize(broker_version=str(LATEST_0_10_0))
    def test_fail_fast_on_incompatible_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, False)

        with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor:
            processor.start()
            monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support CREATE_TOPICS',
                        timeout_sec=60,
                        err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support CREATE_TOPICS' error message " + str(processor.node.account))

        self.kafka.stop()

    @ignore
    @parametrize(broker_version=str(LATEST_0_9))
    @parametrize(broker_version=str(LATEST_0_8_2))
    def test_timeout_on_pre_010_brokers(self, broker_version):
        self.kafka.set_version(KafkaVersion(broker_version))
        self.kafka.start()

        processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, False)

        with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor:
            processor.start()
            monitor.wait_until('Exception in thread "main" org.apache.kafka.streams.errors.BrokerNotFoundException: Could not find any available broker.',
                               timeout_sec=60,
                               err_msg="Never saw 'no available brokers' error message " + str(processor.node.account))

        self.kafka.stop()
Esempio n. 12
0
class QuotaTest(Test):
    """
    These tests verify that quota provides expected functionality -- they run
    producer, broker, and consumer with different clientId and quota configuration and
    check that the observed throughput is close to the value we expect.
    """

    def __init__(self, test_context):
        """:type test_context: ducktape.tests.test.TestContext"""
        super(QuotaTest, self).__init__(test_context=test_context)

        self.topic = 'test_topic'
        self.logger.info('use topic ' + self.topic)

        self.maximum_client_deviation_percentage = 100.0
        self.maximum_broker_deviation_percentage = 5.0
        self.num_records = 50000
        self.record_size = 3000

        self.zk = ZookeeperService(test_context, num_nodes=1)
        self.kafka = KafkaService(test_context, num_nodes=1, zk=self.zk,
                                  security_protocol='SSL', authorizer_class_name='',
                                  interbroker_security_protocol='SSL',
                                  topics={self.topic: {'partitions': 6, 'replication-factor': 1, 'configs': {'min.insync.replicas': 1}}},
                                  jmx_object_names=['kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec',
                                                    'kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec'],
                                  jmx_attributes=['OneMinuteRate'])
        self.num_producers = 1
        self.num_consumers = 2

    def setUp(self):
        self.zk.start()

    def min_cluster_size(self):
        """Override this since we're adding services outside of the constructor"""
        return super(QuotaTest, self).min_cluster_size() + self.num_producers + self.num_consumers

    @cluster(num_nodes=5)
    @matrix(quota_type=[QuotaConfig.CLIENT_ID, QuotaConfig.USER, QuotaConfig.USER_CLIENT], override_quota=[True, False])
    @parametrize(quota_type=QuotaConfig.CLIENT_ID, consumer_num=2)
    @parametrize(quota_type=QuotaConfig.CLIENT_ID, old_broker_throttling_behavior=True)
    @parametrize(quota_type=QuotaConfig.CLIENT_ID, old_client_throttling_behavior=True)
    def test_quota(self, quota_type, override_quota=True, producer_num=1, consumer_num=1,
                   old_broker_throttling_behavior=False, old_client_throttling_behavior=False):
        # Old (pre-2.0) throttling behavior for broker throttles before sending a response to the client.
        if old_broker_throttling_behavior:
            self.kafka.set_version(LATEST_1_1)
        self.kafka.start()

        self.quota_config = QuotaConfig(quota_type, override_quota, self.kafka)
        producer_client_id = self.quota_config.client_id
        consumer_client_id = self.quota_config.client_id

        # Old (pre-2.0) throttling behavior for client does not throttle upon receiving a response with a non-zero throttle time.
        if old_client_throttling_behavior:
            client_version = LATEST_1_1
        else:
            client_version = DEV_BRANCH

        # Produce all messages
        producer = ProducerPerformanceService(
            self.test_context, producer_num, self.kafka,
            topic=self.topic, num_records=self.num_records, record_size=self.record_size, throughput=-1,
            client_id=producer_client_id, version=client_version)

        producer.run()

        # Consume all messages
        consumer = ConsoleConsumer(self.test_context, consumer_num, self.kafka, self.topic,
            consumer_timeout_ms=60000, client_id=consumer_client_id,
            jmx_object_names=['kafka.consumer:type=consumer-fetch-manager-metrics,client-id=%s' % consumer_client_id],
            jmx_attributes=['bytes-consumed-rate'], version=client_version)
        consumer.run()

        for idx, messages in consumer.messages_consumed.iteritems():
            assert len(messages) > 0, "consumer %d didn't consume any message before timeout" % idx

        success, msg = self.validate(self.kafka, producer, consumer)
        assert success, msg

    def validate(self, broker, producer, consumer):
        """
        For each client_id we validate that:
        1) number of consumed messages equals number of produced messages
        2) maximum_producer_throughput <= producer_quota * (1 + maximum_client_deviation_percentage/100)
        3) maximum_broker_byte_in_rate <= producer_quota * (1 + maximum_broker_deviation_percentage/100)
        4) maximum_consumer_throughput <= consumer_quota * (1 + maximum_client_deviation_percentage/100)
        5) maximum_broker_byte_out_rate <= consumer_quota * (1 + maximum_broker_deviation_percentage/100)
        """
        success = True
        msg = ''

        self.kafka.read_jmx_output_all_nodes()

        # validate that number of consumed messages equals number of produced messages
        produced_num = sum([value['records'] for value in producer.results])
        consumed_num = sum([len(value) for value in consumer.messages_consumed.values()])
        self.logger.info('producer produced %d messages' % produced_num)
        self.logger.info('consumer consumed %d messages' % consumed_num)
        if produced_num != consumed_num:
            success = False
            msg += "number of produced messages %d doesn't equal number of consumed messages %d" % (produced_num, consumed_num)

        # validate that maximum_producer_throughput <= producer_quota * (1 + maximum_client_deviation_percentage/100)
        producer_maximum_bps = max(
            metric.value for k, metrics in producer.metrics(group='producer-metrics', name='outgoing-byte-rate', client_id=producer.client_id) for metric in metrics
        )
        producer_quota_bps = self.quota_config.producer_quota
        self.logger.info('producer has maximum throughput %.2f bps with producer quota %.2f bps' % (producer_maximum_bps, producer_quota_bps))
        if producer_maximum_bps > producer_quota_bps*(self.maximum_client_deviation_percentage/100+1):
            success = False
            msg += 'maximum producer throughput %.2f bps exceeded producer quota %.2f bps by more than %.1f%%' % \
                   (producer_maximum_bps, producer_quota_bps, self.maximum_client_deviation_percentage)

        # validate that maximum_broker_byte_in_rate <= producer_quota * (1 + maximum_broker_deviation_percentage/100)
        broker_byte_in_attribute_name = 'kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec:OneMinuteRate'
        broker_maximum_byte_in_bps = broker.maximum_jmx_value[broker_byte_in_attribute_name]
        self.logger.info('broker has maximum byte-in rate %.2f bps with producer quota %.2f bps' %
                         (broker_maximum_byte_in_bps, producer_quota_bps))
        if broker_maximum_byte_in_bps > producer_quota_bps*(self.maximum_broker_deviation_percentage/100+1):
            success = False
            msg += 'maximum broker byte-in rate %.2f bps exceeded producer quota %.2f bps by more than %.1f%%' % \
                   (broker_maximum_byte_in_bps, producer_quota_bps, self.maximum_broker_deviation_percentage)

        # validate that maximum_consumer_throughput <= consumer_quota * (1 + maximum_client_deviation_percentage/100)
        consumer_attribute_name = 'kafka.consumer:type=consumer-fetch-manager-metrics,client-id=%s:bytes-consumed-rate' % consumer.client_id
        consumer_maximum_bps = consumer.maximum_jmx_value[consumer_attribute_name]
        consumer_quota_bps = self.quota_config.consumer_quota
        self.logger.info('consumer has maximum throughput %.2f bps with consumer quota %.2f bps' % (consumer_maximum_bps, consumer_quota_bps))
        if consumer_maximum_bps > consumer_quota_bps*(self.maximum_client_deviation_percentage/100+1):
            success = False
            msg += 'maximum consumer throughput %.2f bps exceeded consumer quota %.2f bps by more than %.1f%%' % \
                   (consumer_maximum_bps, consumer_quota_bps, self.maximum_client_deviation_percentage)

        # validate that maximum_broker_byte_out_rate <= consumer_quota * (1 + maximum_broker_deviation_percentage/100)
        broker_byte_out_attribute_name = 'kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec:OneMinuteRate'
        broker_maximum_byte_out_bps = broker.maximum_jmx_value[broker_byte_out_attribute_name]
        self.logger.info('broker has maximum byte-out rate %.2f bps with consumer quota %.2f bps' %
                         (broker_maximum_byte_out_bps, consumer_quota_bps))
        if broker_maximum_byte_out_bps > consumer_quota_bps*(self.maximum_broker_deviation_percentage/100+1):
            success = False
            msg += 'maximum broker byte-out rate %.2f bps exceeded consumer quota %.2f bps by more than %.1f%%' % \
                   (broker_maximum_byte_out_bps, consumer_quota_bps, self.maximum_broker_deviation_percentage)

        return success, msg