dajac commented on code in PR #17411: URL: https://github.com/apache/kafka/pull/17411#discussion_r1799500109
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java: ########## @@ -378,11 +378,19 @@ private void onErrorResponse(final R response, final long currentTimeMs) { case INVALID_REQUEST: case GROUP_MAX_SIZE_REACHED: case UNSUPPORTED_ASSIGNOR: - case UNSUPPORTED_VERSION: logger.error("{} failed due to {}: {}", heartbeatRequestName(), error, errorMessage); handleFatalFailure(error.exception(errorMessage)); break; + case UNSUPPORTED_VERSION: + String customErrorMessage = "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."; + errorMessage = errorMessage == null || errorMessage.isEmpty() ? customErrorMessage : errorMessage + ". " + customErrorMessage; + logger.error("{} failed due to {}: {}", + heartbeatRequestName(), error, errorMessage); Review Comment: nit: Let's keep it on one line to be consistent with the existing code. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java: ########## @@ -378,11 +378,19 @@ private void onErrorResponse(final R response, final long currentTimeMs) { case INVALID_REQUEST: case GROUP_MAX_SIZE_REACHED: case UNSUPPORTED_ASSIGNOR: - case UNSUPPORTED_VERSION: logger.error("{} failed due to {}: {}", heartbeatRequestName(), error, errorMessage); handleFatalFailure(error.exception(errorMessage)); break; + case UNSUPPORTED_VERSION: + String customErrorMessage = "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."; + errorMessage = errorMessage == null || errorMessage.isEmpty() ? customErrorMessage : errorMessage + ". " + customErrorMessage; Review Comment: I wonder if we should just remove this. It will more or less always return the default error message so it won't look nice for the end user. ########## core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.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"), Review Comment: This one could be removed because it is true by default. ########## core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.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 testAsyncConsumerWithoutConsumerRebalanceProtocol(ClusterInstance clusterInstance) throws Exception { + this.clusterInstance = clusterInstance; Review Comment: nit: This attribute does not seem necessary. Should we just pass `clusterInstance` to `checkUnsupportedConsumerGroupHeartbeat`? ########## core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.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") Review Comment: nit: We use 4 spaces to indent. There are other cases in this file. ########## core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.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 testAsyncConsumerWithoutConsumerRebalanceProtocol(ClusterInstance clusterInstance) throws Exception { + this.clusterInstance = clusterInstance; + checkUnsupportedConsumerGroupHeartbeat(); + } + + @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 testAsyncConsumerWithOldGroupCoordinator(ClusterInstance clusterInstance) throws Exception { + this.clusterInstance = clusterInstance; + checkUnsupportedConsumerGroupHeartbeat(); + } + + private void createTopic(String topic) { + try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + assertDoesNotThrow(() -> admin.createTopics(Collections.singletonList(new NewTopic(topic, 1, (short) 1))).topicId(topic).get()); + } + + assertDoesNotThrow(() -> clusterInstance.waitForTopic(topic, 1)); + } + + private KafkaConsumer<String, String> createAsyncKafkaConsumer(String groupId) { + Map<String, Object> configs = new HashMap<>(); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()); + configs.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name()); + configs.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RangeAssignor.class.getName()); Review Comment: This is actually incorrect when `CONSUMER` is used. You can just remove it. ########## 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: Is `<allow pkg="org.apache.log4j" />` still required with the last changes? ########## core/src/test/java/kafka/clients/consumer/AsyncKafkaConsumerIntegrationTest.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.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 { Review Comment: We actually have `PlaintextConsumerTest` for consumer's integration tests. Any reason why you did not put those there? -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org