You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/02/10 22:58:13 UTC

[GitHub] [kafka] rondagostino opened a new pull request #10105: Kip500 full

rondagostino opened a new pull request #10105:
URL: https://github.com/apache/kafka/pull/10105


   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580402234



##########
File path: build.gradle
##########
@@ -1029,6 +1029,7 @@ project(':metadata') {
     compile project(':clients')
     compile libs.jacksonDatabind
     compile libs.jacksonJDK8Datatypes
+    compile libs.metrics

Review comment:
       thanks. I think this will be a post-2.8 thing




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580575104



##########
File path: tests/kafkatest/services/performance/end_to_end_latency.py
##########
@@ -74,12 +74,15 @@ def __init__(self, context, num_nodes, kafka, topic, num_records, compression_ty
     def start_cmd(self, node):
         args = self.args.copy()
         args.update({
-            'zk_connect': self.kafka.zk_connect_setting(),
             'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
             'config_file': EndToEndLatencyService.CONFIG_FILE,
             'kafka_run_class': self.path.script("kafka-run-class.sh", node),
             'java_class_name': self.java_class_name()
         })
+        if node.version < V_0_9_0_0:

Review comment:
       > name it something like consumer_supports_bootstrap_server... bunch of other bootstrap server functions
   
   Good point.  I renamed it `consumer_supports_bootstrap_server()` as you suggested.
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580486218



##########
File path: tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py
##########
@@ -46,13 +47,16 @@ def __init__(self, test_context):
         self.num_consumers = 1
 
     def setUp(self):
-        self.zk.start()
+        if self.zk:
+            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))
+    @cluster(num_nodes=9)
+    @matrix(broker_version=[str(DEV_BRANCH)], metadata_quorum=quorum.all_non_upgrade)
+    @cluster(num_nodes=9)

Review comment:
       `@cluster` is repeated again




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #10105: Kip500 full

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r579229185



##########
File path: build.gradle
##########
@@ -1029,6 +1029,7 @@ project(':metadata') {
     compile project(':clients')
     compile libs.jacksonDatabind
     compile libs.jacksonJDK8Datatypes
+    compile libs.metrics

Review comment:
       Let's file a JIRA to discuss whether the metadata module should use Kafka metrics or Yammer metrics.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580527682



##########
File path: tests/kafkatest/services/performance/end_to_end_latency.py
##########
@@ -74,12 +74,15 @@ def __init__(self, context, num_nodes, kafka, topic, num_records, compression_ty
     def start_cmd(self, node):
         args = self.args.copy()
         args.update({
-            'zk_connect': self.kafka.zk_connect_setting(),
             'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
             'config_file': EndToEndLatencyService.CONFIG_FILE,
             'kafka_run_class': self.path.script("kafka-run-class.sh", node),
             'java_class_name': self.java_class_name()
         })
+        if node.version < V_0_9_0_0:

Review comment:
       > would be good to have a function in version.py to reflect [functionality unsupported below 0.9.0]
   
   I'll add this function:
   
   ```
       def supports_bootstrap_server(self):
           return self >= V_0_9_0_0
   ```
   
   I then made the changes all over since this 0.9.0 constant is checked in several places:
   
   ```
   tests/kafkatest/services/console_consumer.py
   tests/kafkatest/services/performance/consumer_performance.py
   tests/kafkatest/services/performance/end_to_end_latency.py
   tests/kafkatest/services/performance/producer_performance.py
   tests/kafkatest/tests/core/upgrade_test.py
   tests/kafkatest/tests/core/upgrade_test.py
   ```
   
   We can revert if you think I went too far.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#issuecomment-783624846


   > I wonder if we should disallow tests without `[the @cluster]` annotation in the future.
   
   I think perhaps yes.  Is there a need to allow it?  A simple oversight generates a significant parallelism hit at this point -- and they compound quickly as evidenced here.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe merged pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe merged pull request #10105:
URL: https://github.com/apache/kafka/pull/10105


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on pull request #10105: Kip500 full

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#issuecomment-777517133


   There are 6 failing tests across these 3 test classes:
   
   **kafka.server.KafkaConfigTest**
   ```
   KafkaConfigTest > testFromPropsInvalid() FAILED
       org.opentest4j.AssertionFailedError: Expected exception for property `broker.heartbeat.interval.ms` with invalid value `-1` was not thrown ==> Expected java.lang.Exception to be thrown, but nothing was thrown.
   ```
   
   **AutoTopicCreationManagerTest**
   ```
   AutoTopicCreationManagerTest > testCreateTxnTopic() FAILED
       Wanted but not invoked:
       brokerToControllerChannelManager.sendRequest(
           CreateTopicsRequestData(topics=[CreatableTopic(name='__transaction_state', numPartitions=2, replicationFactor=2, assignments=[], configs=[])], timeoutMs=100, validateOnly=false),
           <any kafka.server.ControllerRequestCompletionHandler>
       );
       -> at kafka.server.AutoTopicCreationManagerTest.testCreateTopic(AutoTopicCreationManagerTest.scala:119)
       Actually, there were zero interactions with this mock.
   
   AutoTopicCreationManagerTest > testCreateOffsetTopic() FAILED
       Wanted but not invoked:
       brokerToControllerChannelManager.sendRequest(
           CreateTopicsRequestData(topics=[CreatableTopic(name='__consumer_offsets', numPartitions=2, replicationFactor=2, assignments=[], configs=[])], timeoutMs=100, validateOnly=false),
           <any kafka.server.ControllerRequestCompletionHandler>
       );
       -> at kafka.server.AutoTopicCreationManagerTest.testCreateTopic(AutoTopicCreationManagerTest.scala:119)
       Actually, there were zero interactions with this mock.
   
   AutoTopicCreationManagerTest > testCreateNonInternalTopic() FAILED
       Wanted but not invoked:
       brokerToControllerChannelManager.sendRequest(
           CreateTopicsRequestData(topics=[CreatableTopic(name='topic', numPartitions=1, replicationFactor=1, assignments=[], configs=[])], timeoutMs=100, validateOnly=false),
           <any kafka.server.ControllerRequestCompletionHandler>
       );
       -> at kafka.server.AutoTopicCreationManagerTest.testCreateTopic(AutoTopicCreationManagerTest.scala:119)
       Actually, there were zero interactions with this mock.
   ```
   
   **MetadataRequestWithForwardingTest**
   ```
   MetadataRequestWithForwardingTest > testAutoCreateOfCollidingTopics() FAILED
       org.opentest4j.AssertionFailedError: expected: <Set(LEADER_NOT_AVAILABLE, INVALID_TOPIC_EXCEPTION)> but was: <Set(UNKNOWN_TOPIC_OR_PARTITION)>
   
   MetadataRequestWithForwardingTest > testAutoTopicCreation() FAILED
       org.opentest4j.AssertionFailedError: expected: <LEADER_NOT_AVAILABLE> but was: <UNKNOWN_TOPIC_OR_PARTITION>
   ```
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580485952



##########
File path: tests/kafkatest/tests/client/client_compatibility_features_test.py
##########
@@ -120,10 +123,9 @@ def invoke_compatibility_program(self, features):
     @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))

Review comment:
       hmm.  we don't want to drop 2.6 and 2.7 here, do we?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580485073



##########
File path: tests/kafkatest/services/performance/end_to_end_latency.py
##########
@@ -74,12 +74,15 @@ def __init__(self, context, num_nodes, kafka, topic, num_records, compression_ty
     def start_cmd(self, node):
         args = self.args.copy()
         args.update({
-            'zk_connect': self.kafka.zk_connect_setting(),
             'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
             'config_file': EndToEndLatencyService.CONFIG_FILE,
             'kafka_run_class': self.path.script("kafka-run-class.sh", node),
             'java_class_name': self.java_class_name()
         })
+        if node.version < V_0_9_0_0:

Review comment:
       It would be good to have a function in `version.py` to reflect this




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580532054



##########
File path: tests/kafkatest/tests/core/round_trip_fault_test.py
##########
@@ -47,32 +55,48 @@ def __init__(self, test_context):
                                      active_topics=active_topics)
 
     def setUp(self):
-        self.zk.start()
+        if self.zk:
+            self.zk.start()
         self.kafka.start()
         self.trogdor.start()
 
     def teardown(self):
         self.trogdor.stop()
         self.kafka.stop()
-        self.zk.stop()
+        if self.zk:
+            self.zk.stop()
 
-    def test_round_trip_workload(self):
+    def remote_quorum_nodes(self):
+        if quorum.for_test(self.test_context) == quorum.zk:
+            return self.zk.nodes
+        elif quorum.for_test(self.test_context) == quorum.remote_raft:
+            return self.kafka.controller_quorum.nodes
+        else: # co-located case, which we currently don't test but handle here for completeness in case we do test it
+            return []

Review comment:
       > throw an exception or does the current code actually work for this case?
   
   The code always needs `<Kafka nodes>` + `<remote quorum nodes>`, where the latter are the ZooKeeper or remote Controller Quorum nodes.  If we were to run this test with co-located Raft Quorum controllers then those nodes would be accounted for because they are part of the Kafka nodes, so there is no need to explicitly add them.  So this code is correct in that it return an empty list for that case.  As was indicated in the comment, it's here just in case we ever decide we want to test it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580485479



##########
File path: tests/kafkatest/tests/client/client_compatibility_features_test.py
##########
@@ -107,7 +108,9 @@ def invoke_compatibility_program(self, features):
           self.logger.info("** Command failed.  See %s for log messages." % ssh_log_file)
           raise
 
-    @parametrize(broker_version=str(DEV_BRANCH))
+    @cluster(num_nodes=7)
+    @matrix(broker_version=[str(DEV_BRANCH)], metadata_quorum=quorum.all_non_upgrade)
+    @cluster(num_nodes=7)

Review comment:
       It seems like `@cluster` is repeated twice here.  We only need it once, right?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580483780



##########
File path: tests/kafkatest/services/console_consumer.py
##########
@@ -151,7 +151,10 @@ def prop_file(self, node):
     def start_cmd(self, node):
         """Return the start command appropriate for the given node."""
         args = self.args.copy()
-        args['zk_connect'] = self.kafka.zk_connect_setting()
+        if self.new_consumer:
+            args['broker_list'] = self.kafka.bootstrap_servers(self.security_config.security_protocol)
+        else:
+            args['zk_connect'] = self.kafka.zk_connect_setting()

Review comment:
       This is a slight behavior change for the old consumer, right?  Previously we set both `broker_list` and `zk_connect`, but now we set only `zk_connect`. Can we keep setting both for now? (I agree we should skip setting zk_connect for the new consumer)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#issuecomment-783579134


   Good work spotting the missing `@cluster` annotations! That is quite a performance win. I wonder if we should disallow tests without this annotation in the future.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580489004



##########
File path: tests/kafkatest/tests/core/round_trip_fault_test.py
##########
@@ -47,32 +55,48 @@ def __init__(self, test_context):
                                      active_topics=active_topics)
 
     def setUp(self):
-        self.zk.start()
+        if self.zk:
+            self.zk.start()
         self.kafka.start()
         self.trogdor.start()
 
     def teardown(self):
         self.trogdor.stop()
         self.kafka.stop()
-        self.zk.stop()
+        if self.zk:
+            self.zk.stop()
 
-    def test_round_trip_workload(self):
+    def remote_quorum_nodes(self):
+        if quorum.for_test(self.test_context) == quorum.zk:
+            return self.zk.nodes
+        elif quorum.for_test(self.test_context) == quorum.remote_raft:
+            return self.kafka.controller_quorum.nodes
+        else: # co-located case, which we currently don't test but handle here for completeness in case we do test it
+            return []

Review comment:
       should this throw an exception? Or does the current code actually work for this case?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580571155



##########
File path: tests/kafkatest/tests/core/round_trip_fault_test.py
##########
@@ -47,32 +55,48 @@ def __init__(self, test_context):
                                      active_topics=active_topics)
 
     def setUp(self):
-        self.zk.start()
+        if self.zk:
+            self.zk.start()
         self.kafka.start()
         self.trogdor.start()
 
     def teardown(self):
         self.trogdor.stop()
         self.kafka.stop()
-        self.zk.stop()
+        if self.zk:
+            self.zk.stop()
 
-    def test_round_trip_workload(self):
+    def remote_quorum_nodes(self):
+        if quorum.for_test(self.test_context) == quorum.zk:
+            return self.zk.nodes
+        elif quorum.for_test(self.test_context) == quorum.remote_raft:
+            return self.kafka.controller_quorum.nodes
+        else: # co-located case, which we currently don't test but handle here for completeness in case we do test it
+            return []

Review comment:
       sounds good




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cmccabe commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580570954



##########
File path: tests/kafkatest/services/performance/end_to_end_latency.py
##########
@@ -74,12 +74,15 @@ def __init__(self, context, num_nodes, kafka, topic, num_records, compression_ty
     def start_cmd(self, node):
         args = self.args.copy()
         args.update({
-            'zk_connect': self.kafka.zk_connect_setting(),
             'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
             'config_file': EndToEndLatencyService.CONFIG_FILE,
             'kafka_run_class': self.path.script("kafka-run-class.sh", node),
             'java_class_name': self.java_class_name()
         })
+        if node.version < V_0_9_0_0:

Review comment:
       thanks.  that sounds good!
   
   can you name it something like `consumer_supports_bootstrap_server`?  There are a bunch of other bootstrap server functions (`acl_command_supports_bootstrap_server`, `topic_command_supports_bootstrap_server`, etc.) so it would be good to be clear




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on a change in pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r580532054



##########
File path: tests/kafkatest/tests/core/round_trip_fault_test.py
##########
@@ -47,32 +55,48 @@ def __init__(self, test_context):
                                      active_topics=active_topics)
 
     def setUp(self):
-        self.zk.start()
+        if self.zk:
+            self.zk.start()
         self.kafka.start()
         self.trogdor.start()
 
     def teardown(self):
         self.trogdor.stop()
         self.kafka.stop()
-        self.zk.stop()
+        if self.zk:
+            self.zk.stop()
 
-    def test_round_trip_workload(self):
+    def remote_quorum_nodes(self):
+        if quorum.for_test(self.test_context) == quorum.zk:
+            return self.zk.nodes
+        elif quorum.for_test(self.test_context) == quorum.remote_raft:
+            return self.kafka.controller_quorum.nodes
+        else: # co-located case, which we currently don't test but handle here for completeness in case we do test it
+            return []

Review comment:
       > throw an exception or does the current code actually work for this case?
   
   The code always needs <Kafka nodes> + <remote quorum nodes>, where the latter are the ZooKeeper or remote Controller Quorum nodes.  If we were to run this test with co-located Raft Quorum controllers then those nodes would be accounted for because they are part of the Kafka nodes, so there is no need to explicitly add them.  So this code is correct in that it return an empty list for that case.  As was indicated in the comment, it's here just in case we ever decide we want to test it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on pull request #10105: MINOR: System tests for Raft-based metadata quorums

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#issuecomment-782646787


   http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2021-02-20--001.1613812911--rondagostino--kip500_full--f59453faa/report.html
   
   Tests|Passes|Failures|Ignored|Time
   940|558|185|197|359 minutes
   
   Some observations.  First, adding the missing @cluster annotations really helped.  Running almost 750 tests in under 6 hours is a dramatic improvement.  Here’s the stats from a run 2 days ago:
   
   Tests|Passes|Failures|Ignored|Time
   729|427|105|197|388 minutes
   
   We are running 743/532 = 40% more tests, but we are able to finish in 10% less time.  So that’s a big win.
   
   We have 130 more tests passing and 80 more tests failing.  It is VERY difficult to find the signal amid so much noise.  A quick browse through our results shows lots of test failures that are not augmented for Raft at all, and many test that are augmented are failing for both ZK and Raft cases.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on a change in pull request #10105: Kip500 full

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #10105:
URL: https://github.com/apache/kafka/pull/10105#discussion_r579407572



##########
File path: build.gradle
##########
@@ -1029,6 +1029,7 @@ project(':metadata') {
     compile project(':clients')
     compile libs.jacksonDatabind
     compile libs.jacksonJDK8Datatypes
+    compile libs.metrics

Review comment:
       > file a JIRA to discuss whether the metadata module should use Kafka metrics or Yammer metrics.
   
   https://issues.apache.org/jira/browse/KAFKA-12348
   
   cc: @cmccabe 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org