AndrewJSchofield commented on code in PR #18850: URL: https://github.com/apache/kafka/pull/18850#discussion_r1959512733
########## clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java: ########## @@ -147,6 +147,17 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals, boolean doLog) } + /** + * Called directly after user configs got parsed (and thus default values is not set). + * This allows to check user's config. + * + * @param parsedValues unmodifiable map of current configuration + * @return a map of updates that should be applied to the configuration (will be validated to prevent bad updates) + */ + protected Map<String, Object> preProcessParsedConfig(Map<String, Object> parsedValues) { + return Collections.emptyMap(); Review Comment: `Map.of()`? ########## clients/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerConfigTest.java: ########## @@ -0,0 +1,54 @@ +/* + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.serialization.StringDeserializer; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class ShareConsumerConfigTest { + + @Test + public void testUnsupportedShareConsumerConfigs() { + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, "1")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "org.apache.kafka.clients.consumer.StickyAssignor")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.clients.consumer.ConsumerInterceptor")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "3000")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 3000)); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 3000)); Review Comment: nit: You have used strings for all numeric configurations except this one. I suggest you choose either string or integer and it in all cases. ########## clients/src/main/java/org/apache/kafka/clients/consumer/ShareConsumerConfig.java: ########## @@ -0,0 +1,77 @@ +/* + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.config.ConfigException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * The share consumer configuration keys + */ +public class ShareConsumerConfig extends ConsumerConfig { + /** + * A list of configuration keys not supported for SHARE consumer. + */ + private static final List<String> SHARE_PROTOCOL_UNSUPPORTED_CONFIGS = List.of( Review Comment: I suggest `SHARE_GROUP_UNSUPPORTED_CONFIGS`. ########## clients/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerConfigTest.java: ########## @@ -0,0 +1,54 @@ +/* + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.serialization.StringDeserializer; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class ShareConsumerConfigTest { + + @Test + public void testUnsupportedShareConsumerConfigs() { + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, "1")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "org.apache.kafka.clients.consumer.StickyAssignor")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.clients.consumer.ConsumerInterceptor")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "3000")); + verifyUnsupportedShareConsumerConfig(Map.of(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 3000)); Review Comment: nit: Repeated line. -- 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