divijvaidya commented on code in PR #14176: URL: https://github.com/apache/kafka/pull/14176#discussion_r1293158075
########## core/src/test/scala/integration/kafka/admin/RemoteTopicCRUDTest.scala: ########## @@ -0,0 +1,262 @@ +/** + * 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.admin + +import kafka.api.IntegrationTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.errors.InvalidConfigurationException +import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig} +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.function.Executable +import org.junit.jupiter.api.{BeforeEach, Tag, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util +import java.util.{Collections, Properties} +import scala.collection.Seq +import scala.concurrent.ExecutionException +import scala.util.Random + +@Tag("integration") +class RemoteTopicCRUDTest extends IntegrationTestHarness { + + val numPartitions = 2 + val numReplicationFactor = 2 + var testTopicName: String = _ + + override protected def brokerCount: Int = 2 + + override protected def modifyConfigs(props: Seq[Properties]): Unit = { + props.foreach(p => p.putAll(overrideProps())) + } + + override protected def kraftControllerConfigs(): Seq[Properties] = { + Seq(overrideProps()) + } + + @BeforeEach + override def setUp(info: TestInfo): Unit = { + super.setUp(info) + testTopicName = s"${info.getTestMethod.get().getName}-${Random.alphanumeric.take(10).mkString}" + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithValidRetentionTime(quorum: String): Unit = { + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200") + topicConfig.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "100") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithValidRetentionSize(quorum: String): Unit = { + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512") + topicConfig.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "256") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInheritedLocalRetentionTime(quorum: String): Unit = { + // inherited local retention ms is 1000 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "1001") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInheritedLocalRetentionSize(quorum: String): Unit = { + // inherited local retention bytes is 1024 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "1025") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInvalidRetentionTime(quorum: String): Unit = { + // inherited local retention ms is 1000 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200") + assertThrowsException(classOf[InvalidConfigurationException], () => + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInvalidRetentionSize(quorum: String): Unit = { + // inherited local retention bytes is 1024 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512") + assertThrowsException(classOf[InvalidConfigurationException], () => + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateCompactedRemoteStorage(quorum: String): Unit = { + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact") + assertThrowsException(classOf[InvalidConfigurationException], () => + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testEnableRemoteLogOnExistingTopicTest(quorum: String): Unit = { + val admin = createAdminClient() + val topicConfig = new Properties() + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + + val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + Collections.singleton( + new AlterConfigOp(new ConfigEntry(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"), + AlterConfigOp.OpType.SET)) + ) + admin.incrementalAlterConfigs(configs).all().get() + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testUpdateTopicConfigWithValidRetentionTimeTest(quorum: String): Unit = { + val admin = createAdminClient() + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + + val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + util.Arrays.asList( + new AlterConfigOp(new ConfigEntry(TopicConfig.RETENTION_MS_CONFIG, "200"), + AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "100"), + AlterConfigOp.OpType.SET) + )) + admin.incrementalAlterConfigs(configs).all().get() + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testUpdateTopicConfigWithValidRetentionSizeTest(quorum: String): Unit = { + val admin = createAdminClient() + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + + val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + util.Arrays.asList( + new AlterConfigOp(new ConfigEntry(TopicConfig.RETENTION_BYTES_CONFIG, "200"), + AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "100"), + AlterConfigOp.OpType.SET) + )) + admin.incrementalAlterConfigs(configs).all().get() + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testUpdateTopicConfigWithInheritedLocalRetentionTime(quorum: String): Unit = { + val admin = createAdminClient() + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) Review Comment: Could be replaced with: ``` createTopic(testTopicName, numPartitions, numReplicationFactor, topicconfig) ``` This will work because KafkaServerTestHarness (parent of IntegrationTestHarness) has this function which do creation of admin client for you. (same comment for other tests) ########## storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java: ########## @@ -454,54 +458,101 @@ public static void validateNames(Properties props) { throw new InvalidConfigurationException("Unknown topic config name: " + name); } + /** + * Validates the values of the given properties. Can be called by both client and server. + * The `props` supplied should contain all the LogConfig properties and the default values are extracted from the + * LogConfig class. + * @param props The properties to be validated + */ public static void validateValues(Map<?, ?> props) { long minCompactionLag = (Long) props.get(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG); long maxCompactionLag = (Long) props.get(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG); if (minCompactionLag > maxCompactionLag) { throw new InvalidConfigurationException("conflict topic config setting " - + TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG + " (" + minCompactionLag + ") > " - + TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG + " (" + maxCompactionLag + ")"); + + TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG + " (" + minCompactionLag + ") > " + + TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG + " (" + maxCompactionLag + ")"); } + } - if (props.containsKey(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG)) { - boolean isRemoteStorageEnabled = (Boolean) props.get(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); - String cleanupPolicy = props.get(TopicConfig.CLEANUP_POLICY_CONFIG).toString().toLowerCase(Locale.getDefault()); - if (isRemoteStorageEnabled && cleanupPolicy.contains(TopicConfig.CLEANUP_POLICY_COMPACT)) { - throw new ConfigException("Remote log storage is unsupported for the compacted topics"); - } + /** + * Validates the default values of the LogConfig. Should be called only by the broker. + * The `props` supplied should contain all the LogConfig properties except + * TopicConfig#REMOTE_LOG_STORAGE_ENABLE_CONFIG and the default values should be extracted from the KafkaConfig. + * @param props The properties to be validated + */ + public static void validateDefaultValuesInBroker(Map<?, ?> props) { + validateValues(props); + Boolean isRemoteLogStorageSystemEnabled = + (Boolean) props.get(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP); + if (isRemoteLogStorageSystemEnabled) { + validateRemoteStorageRetentionSize(props); + validateRemoteStorageRetentionTime(props); + } + } + + /** + * Validates the values of the given properties. Should be called only by the broker. Review Comment: thank you for adding these. very useful! ########## core/src/test/scala/integration/kafka/admin/RemoteTopicCRUDTest.scala: ########## @@ -0,0 +1,262 @@ +/** + * 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.admin + +import kafka.api.IntegrationTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.errors.InvalidConfigurationException +import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig} +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.function.Executable +import org.junit.jupiter.api.{BeforeEach, Tag, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util +import java.util.{Collections, Properties} +import scala.collection.Seq +import scala.concurrent.ExecutionException +import scala.util.Random + +@Tag("integration") +class RemoteTopicCRUDTest extends IntegrationTestHarness { Review Comment: I understand that CRUS is a abbreviation but in Kafka code base we strictly use camel casing. Please rename to RemoteTopicCrudTest ########## core/src/test/scala/integration/kafka/admin/RemoteTopicCRUDTest.scala: ########## @@ -0,0 +1,262 @@ +/** + * 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.admin + +import kafka.api.IntegrationTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.errors.InvalidConfigurationException +import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig} +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.function.Executable +import org.junit.jupiter.api.{BeforeEach, Tag, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util +import java.util.{Collections, Properties} +import scala.collection.Seq +import scala.concurrent.ExecutionException +import scala.util.Random + +@Tag("integration") +class RemoteTopicCRUDTest extends IntegrationTestHarness { + + val numPartitions = 2 + val numReplicationFactor = 2 + var testTopicName: String = _ + + override protected def brokerCount: Int = 2 + + override protected def modifyConfigs(props: Seq[Properties]): Unit = { + props.foreach(p => p.putAll(overrideProps())) + } + + override protected def kraftControllerConfigs(): Seq[Properties] = { + Seq(overrideProps()) + } + + @BeforeEach + override def setUp(info: TestInfo): Unit = { + super.setUp(info) + testTopicName = s"${info.getTestMethod.get().getName}-${Random.alphanumeric.take(10).mkString}" + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithValidRetentionTime(quorum: String): Unit = { + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200") + topicConfig.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, "100") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithValidRetentionSize(quorum: String): Unit = { + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512") + topicConfig.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "256") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInheritedLocalRetentionTime(quorum: String): Unit = { + // inherited local retention ms is 1000 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "1001") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInheritedLocalRetentionSize(quorum: String): Unit = { + // inherited local retention bytes is 1024 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "1025") + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInvalidRetentionTime(quorum: String): Unit = { + // inherited local retention ms is 1000 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200") + assertThrowsException(classOf[InvalidConfigurationException], () => + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateRemoteTopicWithInvalidRetentionSize(quorum: String): Unit = { + // inherited local retention bytes is 1024 + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512") + assertThrowsException(classOf[InvalidConfigurationException], () => + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateCompactedRemoteStorage(quorum: String): Unit = { + val topicConfig = new Properties() + topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact") + assertThrowsException(classOf[InvalidConfigurationException], () => + TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testEnableRemoteLogOnExistingTopicTest(quorum: String): Unit = { + val admin = createAdminClient() + val topicConfig = new Properties() + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + + val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + Collections.singleton( + new AlterConfigOp(new ConfigEntry(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"), + AlterConfigOp.OpType.SET)) + ) + admin.incrementalAlterConfigs(configs).all().get() Review Comment: This validation might not be enough. As an example, if there is a bug in propagating the config to remote log manager, this config change will be a no-op on the server. I will suggest to add: ``` val log = brokers.head.logManager.getLog(tp).get TestUtils.waitUntilTrue(() => { log.config.remoteStorageEnable() }, s"remote storage is not enabled for log with config=${log.config}") ``` (same for other tests) ########## core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala: ########## @@ -383,7 +383,7 @@ abstract class QuorumTestHarness extends Logging { Time.SYSTEM, name = "ZooKeeperTestHarness", new ZKClientConfig) - adminZkClient = new AdminZkClient(zkClient) + adminZkClient = new AdminZkClient(zkClient, None) Review Comment: The default value is already None. Do we need this change? ########## core/src/test/scala/unit/kafka/utils/TestUtils.scala: ########## @@ -561,7 +561,7 @@ object TestUtils extends Logging { replicationFactor: Int = 1, servers: Seq[KafkaBroker], topicConfig: Properties = new Properties): scala.collection.immutable.Map[Int, Int] = { - val adminZkClient = new AdminZkClient(zkClient) + val adminZkClient = new AdminZkClient(zkClient, None) Review Comment: do we need this change? the default parameter is already set to None. (same for other change in this file) ########## core/src/test/scala/integration/kafka/admin/RemoteTopicCRUDTest.scala: ########## @@ -0,0 +1,262 @@ +/** + * 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.admin + +import kafka.api.IntegrationTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.errors.InvalidConfigurationException +import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig} +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.function.Executable +import org.junit.jupiter.api.{BeforeEach, Tag, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util +import java.util.{Collections, Properties} +import scala.collection.Seq +import scala.concurrent.ExecutionException +import scala.util.Random + +@Tag("integration") +class RemoteTopicCRUDTest extends IntegrationTestHarness { Review Comment: It's nice that we added tests for rest of configs but we didn't add the test for the change in this PR! i.e. for a cluster with TS disabled, when an admin client tries to create a topic or alter config for an existing topic to enable TS, it will fail with an error. Please add that. -- 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]
