dajac commented on code in PR #17411:
URL: https://github.com/apache/kafka/pull/17411#discussion_r1798902778
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java:
##########
@@ -379,7 +379,9 @@ private void onErrorResponse(final R response, final long
currentTimeMs) {
case GROUP_MAX_SIZE_REACHED:
case UNSUPPORTED_ASSIGNOR:
case UNSUPPORTED_VERSION:
- logger.error("{} failed due to {}: {}",
heartbeatRequestName(), error, errorMessage);
+ logger.error("{} failed due to {}: {}. The cluster doesn't yet
support the new consumer group protocol." +
+ " Set group.protocol=classic to revert to the classic
protocol until the cluster is upgraded.",
+ heartbeatRequestName(), error, errorMessage);
Review Comment:
This does not sound right to me because it prints the errors for the other
errors too. We should only have it for `UNSUPPORTED_VERSION`.
##########
core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.clients.consumer;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestExtensions;
+import org.apache.kafka.common.utils.LogCaptureAppender;
+
+import org.apache.log4j.Level;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+
+@ExtendWith(ClusterTestExtensions.class)
+public class AsyncKafkaConsumerIntegrationTest {
+ private ClusterInstance clusterInstance;
+
+ @ClusterTest(serverProperties = {
+ @ClusterConfigProperty(key = "offsets.topic.num.partitions", value
= "1"),
+ @ClusterConfigProperty(key = "offsets.topic.replication.factor",
value = "1"),
+ @ClusterConfigProperty(key = "group.coordinator.new.enable", value
= "true"),
+ @ClusterConfigProperty(key =
"group.coordinator.rebalance.protocols", value = "classic")
+ })
+ public void testRebalanceProtocolWithoutConsumer(ClusterInstance
clusterInstance) throws Exception {
Review Comment:
nit: The name of the test is misleading. How about
`testAsyncConsumerWithoutConsumerRebalanceProtocol`?
##########
core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.clients.consumer;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestExtensions;
+import org.apache.kafka.common.utils.LogCaptureAppender;
+
+import org.apache.log4j.Level;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+
+@ExtendWith(ClusterTestExtensions.class)
+public class AsyncKafkaConsumerIntegrationTest {
+ private ClusterInstance clusterInstance;
+
+ @ClusterTest(serverProperties = {
+ @ClusterConfigProperty(key = "offsets.topic.num.partitions", value
= "1"),
+ @ClusterConfigProperty(key = "offsets.topic.replication.factor",
value = "1"),
+ @ClusterConfigProperty(key = "group.coordinator.new.enable", value
= "true"),
+ @ClusterConfigProperty(key =
"group.coordinator.rebalance.protocols", value = "classic")
+ })
+ public void testRebalanceProtocolWithoutConsumer(ClusterInstance
clusterInstance) throws Exception {
+ this.clusterInstance = clusterInstance;
+ String topic = "test-topic";
+ createTopic(topic);
+ try (LogCaptureAppender appender =
LogCaptureAppender.createAndRegister()) {
+ appender.setClassLogger(KafkaConsumer.class, Level.ERROR);
+ KafkaConsumer<String, String> consumer =
createAsyncKafkaConsumer("test-group");
+ consumer.subscribe(Collections.singletonList(topic));
+ TestUtils.waitForCondition(() -> {
+ try {
+ consumer.poll(Duration.ofMillis(1000));
+ return false;
+ } catch (UnsupportedVersionException e) {
+ return true;
+ }
+ }, "Should get UnsupportedVersionException cause of Unsupported
API");
+ TestUtils.waitForCondition(() -> appender.getEvents().stream()
+ .filter(e ->
e.getLevel().equals(Level.ERROR.toString()))
+ .anyMatch(e -> e.getMessage().contains("The
cluster doesn't yet support the new consumer group protocol. " +
+ "Set group.protocol=classic to revert to
the classic protocol until the cluster is upgraded.")),
+ "Error message should contain how to revert to classic
protocol");
Review Comment:
nit: I think that we should rather test that `UnsupportedVersionException`
contains the expected error message.
##########
checkstyle/import-control-core.xml:
##########
@@ -142,4 +142,9 @@
<allow pkg="org.apache.directory" />
<allow pkg="org.apache.mina.core.service" />
</subpackage>
+
+ <subpackage name="clients">
+ <allow pkg="org.apache.kafka.clients" />
+ <allow pkg="org.apache.log4j" />
+ </subpackage>
Review Comment:
Do we really need this?
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java:
##########
@@ -379,7 +379,9 @@ private void onErrorResponse(final R response, final long
currentTimeMs) {
case GROUP_MAX_SIZE_REACHED:
case UNSUPPORTED_ASSIGNOR:
case UNSUPPORTED_VERSION:
- logger.error("{} failed due to {}: {}",
heartbeatRequestName(), error, errorMessage);
+ logger.error("{} failed due to {}: {}. The cluster doesn't yet
support the new consumer group protocol." +
+ " Set group.protocol=classic to revert to the classic
protocol until the cluster is upgraded.",
+ heartbeatRequestName(), error, errorMessage);
handleFatalFailure(error.exception(errorMessage));
Review Comment:
I think that we should also use the custom error message for
`UNSUPPORTED_VERSION` in order to ensure that the end user gets it.
##########
core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.clients.consumer;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestExtensions;
+import org.apache.kafka.common.utils.LogCaptureAppender;
+
+import org.apache.log4j.Level;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+
+@ExtendWith(ClusterTestExtensions.class)
+public class AsyncKafkaConsumerIntegrationTest {
+ private ClusterInstance clusterInstance;
+
+ @ClusterTest(serverProperties = {
+ @ClusterConfigProperty(key = "offsets.topic.num.partitions", value
= "1"),
+ @ClusterConfigProperty(key = "offsets.topic.replication.factor",
value = "1"),
+ @ClusterConfigProperty(key = "group.coordinator.new.enable", value
= "true"),
+ @ClusterConfigProperty(key =
"group.coordinator.rebalance.protocols", value = "classic")
+ })
+ public void testRebalanceProtocolWithoutConsumer(ClusterInstance
clusterInstance) throws Exception {
+ this.clusterInstance = clusterInstance;
+ String topic = "test-topic";
+ createTopic(topic);
+ try (LogCaptureAppender appender =
LogCaptureAppender.createAndRegister()) {
+ appender.setClassLogger(KafkaConsumer.class, Level.ERROR);
+ KafkaConsumer<String, String> consumer =
createAsyncKafkaConsumer("test-group");
+ consumer.subscribe(Collections.singletonList(topic));
+ TestUtils.waitForCondition(() -> {
+ try {
+ consumer.poll(Duration.ofMillis(1000));
+ return false;
+ } catch (UnsupportedVersionException e) {
+ return true;
+ }
+ }, "Should get UnsupportedVersionException cause of Unsupported
API");
+ TestUtils.waitForCondition(() -> appender.getEvents().stream()
+ .filter(e ->
e.getLevel().equals(Level.ERROR.toString()))
+ .anyMatch(e -> e.getMessage().contains("The
cluster doesn't yet support the new consumer group protocol. " +
+ "Set group.protocol=classic to revert to
the classic protocol until the cluster is upgraded.")),
+ "Error message should contain how to revert to classic
protocol");
+ consumer.close();
+ }
+ }
+
+ @ClusterTest(serverProperties = {
+ @ClusterConfigProperty(key = "offsets.topic.num.partitions", value
= "1"),
+ @ClusterConfigProperty(key = "offsets.topic.replication.factor",
value = "1"),
+ @ClusterConfigProperty(key = "group.coordinator.new.enable", value
= "false")
+ })
+ public void testDisableNewGroupCoordiantor(ClusterInstance
clusterInstance) throws Exception {
Review Comment:
nit: `testAsyncConsumerWithOldGroupCoordinator`?
##########
core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.clients.consumer;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestExtensions;
+import org.apache.kafka.common.utils.LogCaptureAppender;
+
+import org.apache.log4j.Level;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+
+@ExtendWith(ClusterTestExtensions.class)
+public class AsyncKafkaConsumerIntegrationTest {
+ private ClusterInstance clusterInstance;
+
+ @ClusterTest(serverProperties = {
+ @ClusterConfigProperty(key = "offsets.topic.num.partitions", value
= "1"),
+ @ClusterConfigProperty(key = "offsets.topic.replication.factor",
value = "1"),
+ @ClusterConfigProperty(key = "group.coordinator.new.enable", value
= "true"),
+ @ClusterConfigProperty(key =
"group.coordinator.rebalance.protocols", value = "classic")
+ })
+ public void testRebalanceProtocolWithoutConsumer(ClusterInstance
clusterInstance) throws Exception {
+ this.clusterInstance = clusterInstance;
+ String topic = "test-topic";
+ createTopic(topic);
+ try (LogCaptureAppender appender =
LogCaptureAppender.createAndRegister()) {
+ appender.setClassLogger(KafkaConsumer.class, Level.ERROR);
+ KafkaConsumer<String, String> consumer =
createAsyncKafkaConsumer("test-group");
+ consumer.subscribe(Collections.singletonList(topic));
+ TestUtils.waitForCondition(() -> {
+ try {
+ consumer.poll(Duration.ofMillis(1000));
+ return false;
+ } catch (UnsupportedVersionException e) {
+ return true;
+ }
+ }, "Should get UnsupportedVersionException cause of Unsupported
API");
+ TestUtils.waitForCondition(() -> appender.getEvents().stream()
+ .filter(e ->
e.getLevel().equals(Level.ERROR.toString()))
+ .anyMatch(e -> e.getMessage().contains("The
cluster doesn't yet support the new consumer group protocol. " +
+ "Set group.protocol=classic to revert to
the classic protocol until the cluster is upgraded.")),
+ "Error message should contain how to revert to classic
protocol");
+ consumer.close();
+ }
+ }
+
+ @ClusterTest(serverProperties = {
+ @ClusterConfigProperty(key = "offsets.topic.num.partitions", value
= "1"),
+ @ClusterConfigProperty(key = "offsets.topic.replication.factor",
value = "1"),
+ @ClusterConfigProperty(key = "group.coordinator.new.enable", value
= "false")
+ })
+ public void testDisableNewGroupCoordiantor(ClusterInstance
clusterInstance) throws Exception {
+ this.clusterInstance = clusterInstance;
+ String topic = "test-topic";
+ createTopic(topic);
+ try (LogCaptureAppender appender =
LogCaptureAppender.createAndRegister()) {
Review Comment:
The body is exactly the same as the previous one. Could we extract it into
an helper method?
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]