From 2b163d9fe6c7a894692b8e4ec64a5c0b554cf178 Mon Sep 17 00:00:00 2001 From: YangKian <1207783292@qq.com> Date: Fri, 26 Apr 2024 13:29:44 +0800 Subject: [PATCH 1/2] enable set server property --- .../scala/hstream/server/KafkaBroker.scala | 8 +++-- .../scala/hstream/server/KafkaConfig.scala | 31 ++++++++++--------- .../hstream/kafka/testing/Utils/Common.java | 3 ++ .../kafka/api/AbstractConsumerTest.scala | 2 +- .../kafka/api/ConsumerTopicCreationTest.scala | 12 ++++--- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../kafka/api/PlaintextConsumerTest.scala | 1 - .../kafka/api/PlaintextProducerSendTest.scala | 25 ++++----------- .../kafka/server/OffsetFetchRequestTest.scala | 2 +- .../scala/utils/kafka/BaseRequestTest.scala | 22 ------------- .../utils/kafka/KafkaServerTestHarness.scala | 26 ++++++---------- 11 files changed, 49 insertions(+), 85 deletions(-) diff --git a/app/src/main/scala/hstream/server/KafkaBroker.scala b/app/src/main/scala/hstream/server/KafkaBroker.scala index d5be2fb..8d6229e 100644 --- a/app/src/main/scala/hstream/server/KafkaBroker.scala +++ b/app/src/main/scala/hstream/server/KafkaBroker.scala @@ -3,6 +3,7 @@ package kafka.server import org.apache.kafka.common.network.ListenerName import org.apache.kafka.metadata.BrokerState import org.apache.kafka.common.utils.Time + import java.nio.file.{Files, Path, Paths, StandardOpenOption} import kafka.utils.Logging import kafka.network.SocketServer @@ -58,10 +59,11 @@ class KafkaBroker( val storeDir = config.testingConfig .getOrElse("store_dir", throw new IllegalArgumentException("store_dir is required")) .asInstanceOf[String] - val extraCommandArgs = - (if (config.autoCreateTopicsEnable) "" else "--disable-auto-create-topic") + val extraProps = config.hstreamKafkaBrokerProperties + .map { case (k, v) => s"--prop $k=$v" } + .mkString(" ") val dockerCmd = - s"docker run -d --network host --name $containerName -v $storeDir:/data/store $image $command $extraCommandArgs" + s"docker run -d --network host --name $containerName -v $storeDir:/data/store $image $command $extraProps" info(s"=> Start hserver by: $dockerCmd") val code = dockerCmd.! if (code != 0) { diff --git a/app/src/main/scala/hstream/server/KafkaConfig.scala b/app/src/main/scala/hstream/server/KafkaConfig.scala index 37e25fe..2a52530 100644 --- a/app/src/main/scala/hstream/server/KafkaConfig.scala +++ b/app/src/main/scala/hstream/server/KafkaConfig.scala @@ -13,23 +13,12 @@ package kafka.server -import java.util -import java.util.{Collections, Properties} -import org.apache.kafka.common.config.{ - AbstractConfig, - ConfigDef, - ConfigException, - ConfigResource, - SaslConfigs, - SecurityConfig, - SslClientAuth, - SslConfigs, - TopicConfig -} +import java.util.Properties +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SecurityConfig, SslClientAuth, SslConfigs, TopicConfig} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.network.ListenerName -import scala.collection.{immutable, Map, Seq} +import scala.collection.{Map, Seq, immutable, mutable} import scala.annotation.nowarn import kafka.cluster.EndPoint import kafka.utils.{CoreUtils, Logging} @@ -55,6 +44,8 @@ object Defaults { // Currently not supported in hstream val NumPartitions = 1 val DefaultReplicationFactor = 1 + // KAFKA_TO_HSTREAM: kafka default value is 3 + val DefaultOffsetsTopicReplicationFactor: Short = 1 // TODO: KAFKA_ORIGINAL // val Listeners = "PLAINTEXT://:9092" @@ -76,6 +67,7 @@ object KafkaConfig { val AutoCreateTopicsEnableProp = "auto.create.topics.enable" val NumPartitionsProp = "num.partitions" val DefaultReplicationFactorProp = "default.replication.factor" + val OffsetsTopicReplicationFactorProp = "offsets.topic.replication.factor" // TODO: KAFKA_ORIGINAL // val ListenersProp = "listeners" @@ -118,6 +110,7 @@ object KafkaConfig { MEDIUM, "$DefaultReplicationFactorDoc" ) + .define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.DefaultOffsetsTopicReplicationFactor, atLeast(1), HIGH, "$OffsetsTopicReplicationFactorDoc") .define(SaslKerberosServiceNameProp, STRING, null, MEDIUM, "$SaslKerberosServiceNameDoc") // TODO: KAFKA_ORIGINAL @@ -194,9 +187,17 @@ class KafkaConfig private ( val advertisedAddress = getString(KafkaConfig.AdvertisedAddressProp) val brokerId = getInt(KafkaConfig.BrokerIdProp) val numPartitions = getInt(KafkaConfig.NumPartitionsProp) - val autoCreateTopicsEnable = getBoolean(KafkaConfig.AutoCreateTopicsEnableProp) val defaultReplicationFactor: Int = getInt(KafkaConfig.DefaultReplicationFactorProp) + def hstreamKafkaBrokerProperties: Map[Any, Any] = { + val props = new mutable.HashMap[Any, Any]() + props.put(KafkaConfig.NumPartitionsProp, getInt(KafkaConfig.NumPartitionsProp)) + props.put(KafkaConfig.DefaultReplicationFactorProp, getInt(KafkaConfig.DefaultReplicationFactorProp)) + props.put(KafkaConfig.AutoCreateTopicsEnableProp, getBoolean(KafkaConfig.AutoCreateTopicsEnableProp)) + props.put(KafkaConfig.OffsetsTopicReplicationFactorProp, getShort(KafkaConfig.OffsetsTopicReplicationFactorProp)) + props + } + // Use advertised listeners if defined, fallback to listeners otherwise def effectiveAdvertisedListeners: Seq[EndPoint] = { val advertisedListenersProp = getString(KafkaConfig.AdvertisedListenersProp) diff --git a/app/src/test/java/io/hstream/kafka/testing/Utils/Common.java b/app/src/test/java/io/hstream/kafka/testing/Utils/Common.java index 88048f8..87b4b94 100644 --- a/app/src/test/java/io/hstream/kafka/testing/Utils/Common.java +++ b/app/src/test/java/io/hstream/kafka/testing/Utils/Common.java @@ -130,6 +130,7 @@ static List> pollUntilAtLeastNumRecords( return records; } + @SuppressWarnings("deprecation") static void pollRecordsUntilTrue( Consumer consumer, Function, Boolean> action, @@ -240,6 +241,7 @@ public static ArrayList runConcurrently(List> runners) { /** * @param pollMs: consumer.poll(pollMs), not total timeout */ + @SuppressWarnings("deprecation") public static List>>> _pollConcurrently( List> consumers, int minTotalRecords, int pollMs) { var totalRecords = new AtomicInteger(0); @@ -280,6 +282,7 @@ public static Map>> pollConcurr /** * @param pollMs: consumer.poll(pollMs), not total timeout */ + @SuppressWarnings("deprecation") public static Map>> pollConcurrentlyWithPollCount( List> consumers, int pollCount, int pollMs) { diff --git a/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index d57e099..eadb197 100644 --- a/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -67,7 +67,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { // TODO HSTREAM // properties.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown - // properties.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + properties.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset // properties.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") // properties.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "100") // set small enough session timeout // properties.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, groupMaxSessionTimeoutMs.toString) diff --git a/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala b/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala index 8e3e3ad..a6cdf52 100644 --- a/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala +++ b/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala @@ -40,7 +40,11 @@ @MethodSource(Array("parameters")) def testAutoTopicCreation(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean): Unit = { val testCase = new ConsumerTopicCreationTest.TestCase(brokerAutoTopicCreationEnable, consumerAllowAutoCreateTopics) - testCase.setUp(new EmptyTestInfo()) + val testInfo = new EmptyTestInfo() { + override def getDisplayName: String = + s"testAutoTopicCreation with args-${brokerAutoTopicCreationEnable.toString},${consumerAllowAutoCreateTopics.toString})" + } + testCase.setUp(testInfo) try testCase.test() finally testCase.tearDown() } @@ -57,7 +61,7 @@ // configure server properties // TODO: ENABLE_FOR_HSTREAM // this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown - // this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, brokerAutoTopicCreationEnable.toString) + this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, brokerAutoTopicCreationEnable.toString) // configure client properties this.producerConfig.setProperty(ProducerConfig.CLIENT_ID_CONFIG, producerClientId) @@ -101,9 +105,7 @@ def parameters: java.util.stream.Stream[Arguments] = { val data = new java.util.ArrayList[Arguments]() - // TODO: ENABLE_FOR_HSTREAM, currently HStream only support set brokerAutoTopicCreationEnable via config file - // for (brokerAutoTopicCreationEnable <- Array(JBoolean.TRUE, JBoolean.FALSE)) - for (brokerAutoTopicCreationEnable <- Array(JBoolean.TRUE)) + for (brokerAutoTopicCreationEnable <- Array(JBoolean.TRUE, JBoolean.FALSE)) for (consumerAutoCreateTopicsPolicy <- Array(JBoolean.TRUE, JBoolean.FALSE)) data.add(Arguments.of(brokerAutoTopicCreationEnable, consumerAutoCreateTopicsPolicy)) data.stream() diff --git a/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index f5aaaa1..8fe6d00 100644 --- a/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -179,7 +179,7 @@ // } } - //TODO: 当前不支持获取 metadataCache + //TODO:KAFKA_TO_HSTREAM: HStream doesn't support get metadataCache // @Test // def testDescribeTopicsWithIds(): Unit = { // client = Admin.create(createConfig) diff --git a/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index b088028..f45424e 100644 --- a/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -748,7 +748,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { } @Test - @Disabled("fix consume reader first") def testFetchOutOfRangeOffsetResetConfigEarliest(): Unit = { this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") // ensure no in-flight fetch request so that the offset can be reset immediately diff --git a/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index f742690..fe9feca 100644 --- a/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -13,31 +13,18 @@ package kafka.api -import java.util.Properties -import java.util.concurrent.{ExecutionException, Future, TimeUnit} import kafka.utils.TestUtils -import org.apache.kafka.clients.producer.{ - BufferExhaustedException, - KafkaProducer, - ProducerConfig, - ProducerRecord, - RecordMetadata -} +import org.apache.kafka.clients.producer._ import org.apache.kafka.common.config.TopicConfig -import org.apache.kafka.common.errors.{ - InvalidTimestampException, - RecordTooLargeException, - SerializationException, - TimeoutException -} +import org.apache.kafka.common.errors.{InvalidTimestampException, RecordTooLargeException, SerializationException, TimeoutException} import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch, Records, TimestampType} import org.apache.kafka.common.serialization.ByteArraySerializer + +import java.util.Properties +import java.util.concurrent.{ExecutionException, Future, TimeUnit} //import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{Disabled, Test} -import org.junit.jupiter.api.Timeout -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{Disabled, Test, Timeout} import java.nio.charset.StandardCharsets diff --git a/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index 500b788..48bb27f 100644 --- a/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -67,7 +67,7 @@ properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) // TODO: ENABLE_FOR_HSTREAM // properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") -// properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") + properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") // properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1") // properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1") // properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1") diff --git a/app/src/test/scala/utils/kafka/BaseRequestTest.scala b/app/src/test/scala/utils/kafka/BaseRequestTest.scala index 881d7c7..144da63 100644 --- a/app/src/test/scala/utils/kafka/BaseRequestTest.scala +++ b/app/src/test/scala/utils/kafka/BaseRequestTest.scala @@ -15,31 +15,9 @@ package kafka.server import kafka.api.IntegrationTestHarness import kafka.network.SocketServer -import kafka.utils.NotNothing -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.Uuid -import org.apache.kafka.common.requests.{ - AbstractRequest, - AbstractResponse, - ApiVersionsRequest, - ApiVersionsResponse, - MetadataRequest, - MetadataResponse, - RequestHeader, - ResponseHeader -} -import org.apache.kafka.common.utils.Utils -import org.apache.kafka.metadata.BrokerState -import java.io.{DataInputStream, DataOutputStream} -import java.net.Socket -import java.nio.ByteBuffer import java.util.Properties -import scala.annotation.nowarn import scala.collection.Seq -import scala.reflect.ClassTag -import scala.jdk.CollectionConverters._ abstract class BaseRequestTest extends IntegrationTestHarness { diff --git a/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala b/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala index 5262d61..d7478b2 100644 --- a/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala +++ b/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala @@ -15,34 +15,26 @@ package kafka.integration import kafka.network.SocketServer - -import java.io.{DataInputStream, DataOutputStream, File} -import java.util.Arrays -import kafka.server.QuorumTestHarness import kafka.server._ -import kafka.utils.{NotNothing, TestUtils} -import org.apache.kafka.common.security.auth.SecurityProtocol -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} - -import scala.collection.{mutable, Seq} -import scala.jdk.CollectionConverters._ -import java.util.Properties import kafka.utils.TestUtils.{createAdminClient, resource} -import org.apache.kafka.common.acl.AccessControlEntry -import org.apache.kafka.common.{KafkaException, Uuid} +import kafka.utils.{NotNothing, TestUtils} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, ApiVersionsRequest, ApiVersionsResponse, MetadataRequest, MetadataResponse, RequestHeader, ResponseHeader} -import org.apache.kafka.common.resource.ResourcePattern -import org.apache.kafka.common.security.scram.ScramCredential +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.common.{KafkaException, Uuid} import org.apache.kafka.metadata.BrokerState +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import java.io.{DataInputStream, DataOutputStream, File} import java.net.Socket import java.nio.ByteBuffer +import java.util.{Arrays, Properties} import scala.annotation.nowarn +import scala.collection.{Seq, mutable} +import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag -import java.nio.file.{Path, Paths} // import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT /** From 0dbd3758d5cab04699491e64055efe315bda1fc6 Mon Sep 17 00:00:00 2001 From: YangKian <1207783292@qq.com> Date: Fri, 26 Apr 2024 13:42:36 +0800 Subject: [PATCH 2/2] add scalafmt --- app/build.gradle | 5 + .../scala/hstream/network/SocketServer.scala | 23 +- .../scala/hstream/server/KafkaBroker.scala | 22 +- .../scala/hstream/server/KafkaConfig.scala | 46 +- .../main/scala/kafka/cluster/EndPoint.scala | 36 +- .../main/scala/kafka/utils/Implicits.scala | 23 +- app/src/main/scala/kafka/utils/Json.scala | 29 +- .../main/scala/kafka/utils/NotNothing.scala | 55 +- .../scala/kafka/utils/json/DecodeJson.scala | 32 +- .../scala/kafka/utils/json/JsonArray.scala | 7 +- .../scala/kafka/utils/json/JsonObject.scala | 7 +- .../scala/kafka/utils/json/JsonValue.scala | 41 +- app/src/main/scala/utils/CoreUtils.scala | 145 +- app/src/main/scala/utils/Logging.scala | 18 +- .../admin/ListOffsetsIntegrationTest.scala | 40 +- .../kafka/api/AbstractConsumerTest.scala | 2 +- .../kafka/api/BaseAdminIntegrationTest.scala | 415 ++--- .../kafka/api/BaseAsyncConsumerTest.scala | 11 +- .../kafka/api/BaseProducerSendTest.scala | 23 +- .../kafka/api/ConsumerTopicCreationTest.scala | 230 +-- .../api/PlaintextAdminIntegrationTest.scala | 1520 +++++++++-------- .../kafka/api/PlaintextConsumerTest.scala | 22 +- .../kafka/api/PlaintextProducerSendTest.scala | 30 +- .../kafka/api/ProducerCompressionTest.scala | 23 +- .../api/ProducerFailureHandlingTest.scala | 23 +- .../api/ProducerSendWhileDeletionTest.scala | 22 +- .../AbstractCreateTopicsRequestTest.scala | 23 +- .../server/CreateTopicsRequestTest.scala | 23 +- .../kafka/server/OffsetFetchRequestTest.scala | 531 +++--- .../kafka/server/ProduceRequestTest.scala | 23 +- .../utils/kafka/BaseFetchRequestTest.scala | 22 +- .../scala/utils/kafka/BaseRequestTest.scala | 23 +- .../utils/kafka/IntegrationTestHarness.scala | 57 +- .../utils/kafka/KafkaServerTestHarness.scala | 68 +- .../scala/utils/kafka/QuorumTestHarness.scala | 30 +- .../utils/kafka/utils/JaasTestUtils.scala | 24 +- .../utils/kafka/utils/TestInfoUtils.scala | 4 +- .../scala/utils/kafka/utils/TestUtils.scala | 55 +- java.header | 16 + 39 files changed, 2055 insertions(+), 1694 deletions(-) create mode 100644 java.header diff --git a/app/build.gradle b/app/build.gradle index 2584021..1ef397b 100644 --- a/app/build.gradle +++ b/app/build.gradle @@ -123,6 +123,11 @@ spotless { java { googleJavaFormat() } + scala { + target '**/*.scala' + scalafmt('3.8.0').configFile('../.scalafmt.conf') + licenseHeaderFile '../java.header', 'package' + } groovyGradle { target '*.gradle' diff --git a/app/src/main/scala/hstream/network/SocketServer.scala b/app/src/main/scala/hstream/network/SocketServer.scala index b56836f..87f0ebe 100644 --- a/app/src/main/scala/hstream/network/SocketServer.scala +++ b/app/src/main/scala/hstream/network/SocketServer.scala @@ -1,3 +1,19 @@ +/* + * 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.network import org.apache.kafka.common.network.ListenerName @@ -7,9 +23,10 @@ import kafka.server.KafkaConfig class SocketServer(val config: KafkaConfig) { def boundPort(listenerName: ListenerName): Int = { - val listener = config.effectiveAdvertisedListeners.find(_.listenerName == listenerName).getOrElse( - sys.error(s"Could not find listener with name ${listenerName.value}")) - listener.port + val listener = config.effectiveAdvertisedListeners + .find(_.listenerName == listenerName) + .getOrElse(sys.error(s"Could not find listener with name ${listenerName.value}")) + listener.port } } diff --git a/app/src/main/scala/hstream/server/KafkaBroker.scala b/app/src/main/scala/hstream/server/KafkaBroker.scala index 8d6229e..d21a67c 100644 --- a/app/src/main/scala/hstream/server/KafkaBroker.scala +++ b/app/src/main/scala/hstream/server/KafkaBroker.scala @@ -1,3 +1,19 @@ +/* + * 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.server import org.apache.kafka.common.network.ListenerName @@ -59,9 +75,9 @@ class KafkaBroker( val storeDir = config.testingConfig .getOrElse("store_dir", throw new IllegalArgumentException("store_dir is required")) .asInstanceOf[String] - val extraProps = config.hstreamKafkaBrokerProperties - .map { case (k, v) => s"--prop $k=$v" } - .mkString(" ") + val extraProps = config.hstreamKafkaBrokerProperties + .map { case (k, v) => s"--prop $k=$v" } + .mkString(" ") val dockerCmd = s"docker run -d --network host --name $containerName -v $storeDir:/data/store $image $command $extraProps" info(s"=> Start hserver by: $dockerCmd") diff --git a/app/src/main/scala/hstream/server/KafkaConfig.scala b/app/src/main/scala/hstream/server/KafkaConfig.scala index 2a52530..5a4022a 100644 --- a/app/src/main/scala/hstream/server/KafkaConfig.scala +++ b/app/src/main/scala/hstream/server/KafkaConfig.scala @@ -1,24 +1,37 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.server import java.util.Properties -import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SecurityConfig, SslClientAuth, SslConfigs, TopicConfig} +import org.apache.kafka.common.config.{ + AbstractConfig, + ConfigDef, + ConfigException, + ConfigResource, + SaslConfigs, + SecurityConfig, + SslClientAuth, + SslConfigs, + TopicConfig +} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.network.ListenerName -import scala.collection.{Map, Seq, immutable, mutable} +import scala.collection.{immutable, mutable, Map, Seq} import scala.annotation.nowarn import kafka.cluster.EndPoint import kafka.utils.{CoreUtils, Logging} @@ -110,7 +123,14 @@ object KafkaConfig { MEDIUM, "$DefaultReplicationFactorDoc" ) - .define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.DefaultOffsetsTopicReplicationFactor, atLeast(1), HIGH, "$OffsetsTopicReplicationFactorDoc") + .define( + OffsetsTopicReplicationFactorProp, + SHORT, + Defaults.DefaultOffsetsTopicReplicationFactor, + atLeast(1), + HIGH, + "$OffsetsTopicReplicationFactorDoc" + ) .define(SaslKerberosServiceNameProp, STRING, null, MEDIUM, "$SaslKerberosServiceNameDoc") // TODO: KAFKA_ORIGINAL diff --git a/app/src/main/scala/kafka/cluster/EndPoint.scala b/app/src/main/scala/kafka/cluster/EndPoint.scala index 89c9f5e..63cb08c 100644 --- a/app/src/main/scala/kafka/cluster/EndPoint.scala +++ b/app/src/main/scala/kafka/cluster/EndPoint.scala @@ -1,12 +1,12 @@ /* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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, @@ -14,10 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.cluster -import org.apache.kafka.common.{KafkaException, Endpoint => JEndpoint} +import org.apache.kafka.common.{Endpoint => JEndpoint, KafkaException} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Utils @@ -36,17 +35,22 @@ object EndPoint { * Create EndPoint object from `connectionString` and optional `securityProtocolMap`. If the latter is not provided, * we fallback to the default behaviour where listener names are the same as security protocols. * - * @param connectionString the format is listener_name://host:port or listener_name://[ipv6 host]:port - * for example: PLAINTEXT://myhost:9092, CLIENT://myhost:9092 or REPLICATION://[::1]:9092 - * Host can be empty (PLAINTEXT://:9092) in which case we'll bind to default interface - * Negative ports are also accepted, since they are used in some unit tests + * @param connectionString + * the format is listener_name://host:port or listener_name://[ipv6 host]:port for example: PLAINTEXT://myhost:9092, + * CLIENT://myhost:9092 or REPLICATION://[::1]:9092 Host can be empty (PLAINTEXT://:9092) in which case we'll bind + * to default interface Negative ports are also accepted, since they are used in some unit tests */ - def createEndPoint(connectionString: String, securityProtocolMap: Option[Map[ListenerName, SecurityProtocol]]): EndPoint = { + def createEndPoint( + connectionString: String, + securityProtocolMap: Option[Map[ListenerName, SecurityProtocol]] + ): EndPoint = { val protocolMap = securityProtocolMap.getOrElse(DefaultSecurityProtocolMap) def securityProtocol(listenerName: ListenerName): SecurityProtocol = - protocolMap.getOrElse(listenerName, - throw new IllegalArgumentException(s"No security protocol defined for listener ${listenerName.value}")) + protocolMap.getOrElse( + listenerName, + throw new IllegalArgumentException(s"No security protocol defined for listener ${listenerName.value}") + ) connectionString match { case uriParseExp(listenerNameString, "", port) => @@ -67,10 +71,12 @@ object EndPoint { } def fromJava(endpoint: JEndpoint): EndPoint = - new EndPoint(endpoint.host(), + new EndPoint( + endpoint.host(), endpoint.port(), new ListenerName(endpoint.listenerName().get()), - endpoint.securityProtocol()) + endpoint.securityProtocol() + ) } /** @@ -80,7 +86,7 @@ case class EndPoint(host: String, port: Int, listenerName: ListenerName, securit def connectionString: String = { val hostport = if (host == null) - ":"+port + ":" + port else Utils.formatAddress(host, port) listenerName.value + "://" + hostport diff --git a/app/src/main/scala/kafka/utils/Implicits.scala b/app/src/main/scala/kafka/utils/Implicits.scala index fbd22ec..e6d3e20 100644 --- a/app/src/main/scala/kafka/utils/Implicits.scala +++ b/app/src/main/scala/kafka/utils/Implicits.scala @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.utils import java.util @@ -24,16 +23,15 @@ import scala.annotation.nowarn import scala.jdk.CollectionConverters._ /** - * In order to have these implicits in scope, add the following import: - * - * `import kafka.utils.Implicits._` - */ + * In order to have these implicits in scope, add the following import: + * + * `import kafka.utils.Implicits._` + */ object Implicits { /** - * The java.util.Properties.putAll override introduced in Java 9 is seen as an overload by the - * Scala compiler causing ambiguity errors in some cases. The `++=` methods introduced via - * implicits provide a concise alternative. + * The java.util.Properties.putAll override introduced in Java 9 is seen as an overload by the Scala compiler causing + * ambiguity errors in some cases. The `++=` methods introduced via implicits provide a concise alternative. * * See https://github.com/scala/bug/issues/10418 for more details. */ @@ -48,12 +46,11 @@ object Implicits { } /** - * Exposes `forKeyValue` which maps to `foreachEntry` in Scala 2.13 and `foreach` in Scala 2.12 - * (with the help of scala.collection.compat). `foreachEntry` avoids the tuple allocation and - * is more efficient. + * Exposes `forKeyValue` which maps to `foreachEntry` in Scala 2.13 and `foreach` in Scala 2.12 (with the help of + * scala.collection.compat). `foreachEntry` avoids the tuple allocation and is more efficient. * - * This was not named `foreachEntry` to avoid `unused import` warnings in Scala 2.13 (the implicit - * would not be triggered in Scala 2.13 since `Map.foreachEntry` would have precedence). + * This was not named `foreachEntry` to avoid `unused import` warnings in Scala 2.13 (the implicit would not be + * triggered in Scala 2.13 since `Map.foreachEntry` would have precedence). */ @nowarn("cat=unused-imports") implicit class MapExtensionMethods[K, V](private val self: scala.collection.Map[K, V]) extends AnyVal { diff --git a/app/src/main/scala/kafka/utils/Json.scala b/app/src/main/scala/kafka/utils/Json.scala index 049941c..c73c03a 100644 --- a/app/src/main/scala/kafka/utils/Json.scala +++ b/app/src/main/scala/kafka/utils/Json.scala @@ -1,11 +1,11 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 - * + * 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 @@ -36,8 +36,7 @@ object Json { def parseFull(input: String): Option[JsonValue] = tryParseFull(input).toOption /** - * Parse a JSON string into either a generic type T, or a JsonProcessingException in the case of - * exception. + * Parse a JSON string into either a generic type T, or a JsonProcessingException in the case of exception. */ def parseStringAs[T](input: String)(implicit tag: ClassTag[T]): Either[JsonProcessingException, T] = { try Right(mapper.readValue(input, tag.runtimeClass).asInstanceOf[T]) @@ -65,9 +64,11 @@ object Json { /** * Parse a JSON string into a JsonValue if possible. It returns an `Either` where `Left` will be an exception and - * `Right` is the `JsonValue`. - * @param input a JSON string to parse - * @return An `Either` which in case of `Left` means an exception and `Right` is the actual return value. + * `Right` is the `JsonValue`. + * @param input + * a JSON string to parse + * @return + * An `Either` which in case of `Left` means an exception and `Right` is the actual return value. */ def tryParseFull(input: String): Either[JsonProcessingException, JsonValue] = if (input == null || input.isEmpty) @@ -77,16 +78,16 @@ object Json { catch { case e: JsonProcessingException => Left(e) } /** - * Encode an object into a JSON string. This method accepts any type supported by Jackson's ObjectMapper in - * the default configuration. That is, Java collections are supported, but Scala collections are not (to avoid - * a jackson-scala dependency). + * Encode an object into a JSON string. This method accepts any type supported by Jackson's ObjectMapper in the + * default configuration. That is, Java collections are supported, but Scala collections are not (to avoid a + * jackson-scala dependency). */ def encodeAsString(obj: Any): String = mapper.writeValueAsString(obj) /** * Encode an object into a JSON value in bytes. This method accepts any type supported by Jackson's ObjectMapper in - * the default configuration. That is, Java collections are supported, but Scala collections are not (to avoid - * a jackson-scala dependency). + * the default configuration. That is, Java collections are supported, but Scala collections are not (to avoid a + * jackson-scala dependency). */ def encodeAsBytes(obj: Any): Array[Byte] = mapper.writeValueAsBytes(obj) } diff --git a/app/src/main/scala/kafka/utils/NotNothing.scala b/app/src/main/scala/kafka/utils/NotNothing.scala index aee345e..28cb0e0 100644 --- a/app/src/main/scala/kafka/utils/NotNothing.scala +++ b/app/src/main/scala/kafka/utils/NotNothing.scala @@ -1,36 +1,35 @@ -/** - * 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. - */ - +/* + * 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.utils import scala.annotation.implicitNotFound /** - * This is a trick to prevent the compiler from inferring the `Nothing` type in cases where it would be a bug to do - * so. An example is the following method: - * - * ``` - * def body[T <: AbstractRequest](implicit classTag: ClassTag[T], nn: NotNothing[T]): T - * ``` - * - * If we remove the `nn` parameter and we invoke it without any type parameters (e.g. `request.body`), `Nothing` would - * be inferred, which is not desirable. As defined above, we get a helpful compiler error asking the user to provide - * the type parameter explicitly. - */ + * This is a trick to prevent the compiler from inferring the `Nothing` type in cases where it would be a bug to do so. + * An example is the following method: + * + * ``` + * def body[T <: AbstractRequest](implicit classTag: ClassTag[T], nn: NotNothing[T]): T + * ``` + * + * If we remove the `nn` parameter and we invoke it without any type parameters (e.g. `request.body`), `Nothing` would + * be inferred, which is not desirable. As defined above, we get a helpful compiler error asking the user to provide the + * type parameter explicitly. + */ @implicitNotFound("Unable to infer type parameter, please provide it explicitly.") trait NotNothing[T] diff --git a/app/src/main/scala/kafka/utils/json/DecodeJson.scala b/app/src/main/scala/kafka/utils/json/DecodeJson.scala index 71bfd61..6d17d45 100644 --- a/app/src/main/scala/kafka/utils/json/DecodeJson.scala +++ b/app/src/main/scala/kafka/utils/json/DecodeJson.scala @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.utils.json import scala.collection.{Map, Seq} @@ -37,19 +36,20 @@ trait DecodeJson[T] { /** * Decode the JSON node provided into an instance of `T`. * - * @throws JsonMappingException if `node` cannot be decoded into `T`. + * @throws JsonMappingException + * if `node` cannot be decoded into `T`. */ def decode(node: JsonNode): T = decodeEither(node) match { case Right(x) => x - case Left(x) => throw new JsonMappingException(null, x) + case Left(x) => throw new JsonMappingException(null, x) } } /** - * Contains `DecodeJson` type class instances. That is, we need one instance for each type that we want to be able to - * to parse into. It is a compiler error to try to parse into a type for which there is no instance. + * Contains `DecodeJson` type class instances. That is, we need one instance for each type that we want to be able to to + * parse into. It is a compiler error to try to parse into a type for which there is no instance. */ object DecodeJson { @@ -85,24 +85,32 @@ object DecodeJson { else decodeJson.decodeEither(node).map(Some(_)) } - implicit def decodeSeq[E, S[+T] <: Seq[E]](implicit decodeJson: DecodeJson[E], factory: Factory[E, S[E]]): DecodeJson[S[E]] = (node: JsonNode) => { + implicit def decodeSeq[E, S[+T] <: Seq[E]](implicit + decodeJson: DecodeJson[E], + factory: Factory[E, S[E]] + ): DecodeJson[S[E]] = (node: JsonNode) => { if (node.isArray) decodeIterator(node.elements.asScala)(decodeJson.decodeEither) else Left(s"Expected JSON array, received $node") } - implicit def decodeMap[V, M[K, +V] <: Map[K, V]](implicit decodeJson: DecodeJson[V], factory: Factory[(String, V), M[String, V]]): DecodeJson[M[String, V]] = (node: JsonNode) => { + implicit def decodeMap[V, M[K, +V] <: Map[K, V]](implicit + decodeJson: DecodeJson[V], + factory: Factory[(String, V), M[String, V]] + ): DecodeJson[M[String, V]] = (node: JsonNode) => { if (node.isObject) decodeIterator(node.fields.asScala)(e => decodeJson.decodeEither(e.getValue).map(v => (e.getKey, v))) else Left(s"Expected JSON object, received $node") } - private def decodeIterator[S, T, C](it: Iterator[S])(f: S => Either[String, T])(implicit factory: Factory[T, C]): Either[String, C] = { + private def decodeIterator[S, T, C]( + it: Iterator[S] + )(f: S => Either[String, T])(implicit factory: Factory[T, C]): Either[String, C] = { val result = factory.newBuilder while (it.hasNext) { f(it.next()) match { case Right(x) => result += x - case Left(x) => return Left(x) + case Left(x) => return Left(x) } } Right(result.result()) diff --git a/app/src/main/scala/kafka/utils/json/JsonArray.scala b/app/src/main/scala/kafka/utils/json/JsonArray.scala index c22eda8..062d486 100644 --- a/app/src/main/scala/kafka/utils/json/JsonArray.scala +++ b/app/src/main/scala/kafka/utils/json/JsonArray.scala @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.utils.json import scala.collection.Iterator diff --git a/app/src/main/scala/kafka/utils/json/JsonObject.scala b/app/src/main/scala/kafka/utils/json/JsonObject.scala index 9bf91ae..1336d25 100644 --- a/app/src/main/scala/kafka/utils/json/JsonObject.scala +++ b/app/src/main/scala/kafka/utils/json/JsonObject.scala @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.utils.json import com.fasterxml.jackson.databind.JsonMappingException diff --git a/app/src/main/scala/kafka/utils/json/JsonValue.scala b/app/src/main/scala/kafka/utils/json/JsonValue.scala index ff62c6c..5335d47 100644 --- a/app/src/main/scala/kafka/utils/json/JsonValue.scala +++ b/app/src/main/scala/kafka/utils/json/JsonValue.scala @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,15 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.utils.json import com.fasterxml.jackson.databind.{JsonMappingException, JsonNode} import com.fasterxml.jackson.databind.node.{ArrayNode, ObjectNode} /** - * A simple wrapper over Jackson's JsonNode that enables type safe parsing via the `DecodeJson` type - * class. + * A simple wrapper over Jackson's JsonNode that enables type safe parsing via the `DecodeJson` type class. * * Typical usage would be something like: * @@ -45,13 +43,14 @@ trait JsonValue { /** * Decode this JSON value into an instance of `T`. * - * @throws JsonMappingException if this value cannot be decoded into `T`. + * @throws JsonMappingException + * if this value cannot be decoded into `T`. */ def to[T](implicit decodeJson: DecodeJson[T]): T = decodeJson.decode(node) /** - * Decode this JSON value into an instance of `Right[T]`, if possible. Otherwise, return an error message - * wrapped by an instance of `Left`. + * Decode this JSON value into an instance of `Right[T]`, if possible. Otherwise, return an error message wrapped by + * an instance of `Left`. */ def toEither[T](implicit decodeJson: DecodeJson[T]): Either[String, T] = decodeJson.decodeEither(node) @@ -66,10 +65,11 @@ trait JsonValue { */ def asJsonObjectOption: Option[JsonObject] = this match { case j: JsonObject => Some(j) - case _ => node match { - case n: ObjectNode => Some(new JsonObject(n)) - case _ => None - } + case _ => + node match { + case n: ObjectNode => Some(new JsonObject(n)) + case _ => None + } } /** @@ -83,17 +83,18 @@ trait JsonValue { */ def asJsonArrayOption: Option[JsonArray] = this match { case j: JsonArray => Some(j) - case _ => node match { - case n: ArrayNode => Some(new JsonArray(n)) - case _ => None - } + case _ => + node match { + case n: ArrayNode => Some(new JsonArray(n)) + case _ => None + } } override def hashCode: Int = node.hashCode override def equals(a: Any): Boolean = a match { case a: JsonValue => node == a.node - case _ => false + case _ => false } override def toString: String = node.toString @@ -107,8 +108,8 @@ object JsonValue { */ def apply(node: JsonNode): JsonValue = node match { case n: ObjectNode => new JsonObject(n) - case n: ArrayNode => new JsonArray(n) - case _ => new BasicJsonValue(node) + case n: ArrayNode => new JsonArray(n) + case _ => new BasicJsonValue(node) } private class BasicJsonValue private[json] (protected val node: JsonNode) extends JsonValue diff --git a/app/src/main/scala/utils/CoreUtils.scala b/app/src/main/scala/utils/CoreUtils.scala index 3aca6a5..d147785 100644 --- a/app/src/main/scala/utils/CoreUtils.scala +++ b/app/src/main/scala/utils/CoreUtils.scala @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.utils import java.io._ @@ -27,7 +26,7 @@ import com.typesafe.scalalogging.Logger import javax.management._ import scala.collection._ -import scala.collection.{Seq, mutable} +import scala.collection.{mutable, Seq} import kafka.cluster.EndPoint import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol @@ -43,9 +42,8 @@ import scala.annotation.nowarn * the standard library etc. * * If you are making a new helper function and want to add it to this class please ensure the following: - * 1. It has documentation - * 2. It is the most general possible utility, not just the thing you needed in one particular place - * 3. You have tests for it if it is nontrivial in any way + * 1. It has documentation 2. It is the most general possible utility, not just the thing you needed in one particular + * place 3. You have tests for it if it is nontrivial in any way */ object CoreUtils { private val logger = Logger(getClass) @@ -57,29 +55,34 @@ object CoreUtils { if (iterable.isEmpty) ifEmpty else iterable.min(cmp) /** - * Do the given action and log any exceptions thrown without rethrowing them. - * - * @param action The action to execute. - * @param logging The logging instance to use for logging the thrown exception. - * @param logLevel The log level to use for logging. - */ + * Do the given action and log any exceptions thrown without rethrowing them. + * + * @param action + * The action to execute. + * @param logging + * The logging instance to use for logging the thrown exception. + * @param logLevel + * The log level to use for logging. + */ def swallow(action: => Unit, logging: Logging, logLevel: Level = Level.WARN): Unit = { try { action } catch { - case e: Throwable => logLevel match { - case Level.ERROR => logger.error(e.getMessage, e) - case Level.WARN => logger.warn(e.getMessage, e) - case Level.INFO => logger.info(e.getMessage, e) - case Level.DEBUG => logger.debug(e.getMessage, e) - case Level.TRACE => logger.trace(e.getMessage, e) - } + case e: Throwable => + logLevel match { + case Level.ERROR => logger.error(e.getMessage, e) + case Level.WARN => logger.warn(e.getMessage, e) + case Level.INFO => logger.info(e.getMessage, e) + case Level.DEBUG => logger.debug(e.getMessage, e) + case Level.TRACE => logger.trace(e.getMessage, e) + } } } /** * Recursively delete the list of files/directories and any subfiles (if any exist) - * @param files sequence of files to be deleted + * @param files + * sequence of files to be deleted */ def delete(files: Seq[String]): Unit = files.foreach(f => Utils.delete(new File(f))) @@ -108,14 +111,15 @@ object CoreUtils { } /** - * Register the given mbean with the platform mbean server, - * unregistering any mbean that was there before. Note, - * this method will not throw an exception if the registration - * fails (since there is nothing you can do and it isn't fatal), - * instead it just returns false indicating the registration failed. - * @param mbean The object to register as an mbean - * @param name The name to register this mbean with - * @return true if the registration succeeded + * Register the given mbean with the platform mbean server, unregistering any mbean that was there before. Note, this + * method will not throw an exception if the registration fails (since there is nothing you can do and it isn't + * fatal), instead it just returns false indicating the registration failed. + * @param mbean + * The object to register as an mbean + * @param name + * The name to register this mbean with + * @return + * true if the registration succeeded */ def registerMBean(mbean: Object, name: String): Boolean = { try { @@ -136,7 +140,8 @@ object CoreUtils { /** * Unregister the mbean with the given name, if there is one registered - * @param name The mbean name to unregister + * @param name + * The mbean name to unregister */ def unregisterMBean(name: String): Unit = { val mbs = ManagementFactory.getPlatformMBeanServer() @@ -148,36 +153,37 @@ object CoreUtils { } /** - * Read some bytes into the provided buffer, and return the number of bytes read. If the - * channel has been closed or we get -1 on the read for any reason, throw an EOFException + * Read some bytes into the provided buffer, and return the number of bytes read. If the channel has been closed or we + * get -1 on the read for any reason, throw an EOFException */ def read(channel: ReadableByteChannel, buffer: ByteBuffer): Int = { channel.read(buffer) match { case -1 => throw new EOFException("Received -1 when reading from channel, socket has likely been closed.") - case n => n + case n => n } } /** - * This method gets comma separated values which contains key,value pairs and returns a map of - * key value pairs. the format of allCSVal is key1:val1, key2:val2 .... - * Also supports strings with multiple ":" such as IpV6 addresses, taking the last occurrence - * of the ":" in the pair as the split, eg a:b:c:val1, d:e:f:val2 => a:b:c -> val1, d:e:f -> val2 + * This method gets comma separated values which contains key,value pairs and returns a map of key value pairs. the + * format of allCSVal is key1:val1, key2:val2 .... Also supports strings with multiple ":" such as IpV6 addresses, + * taking the last occurrence of the ":" in the pair as the split, eg a:b:c:val1, d:e:f:val2 => a:b:c -> val1, d:e:f + * -> val2 */ def parseCsvMap(str: String): Map[String, String] = { val map = new mutable.HashMap[String, String] if ("".equals(str)) return map - val keyVals = str.split("\\s*,\\s*").map(s => { - val lio = s.lastIndexOf(":") - (s.substring(0,lio).trim, s.substring(lio + 1).trim) - }) + val keyVals = str + .split("\\s*,\\s*") + .map(s => { + val lio = s.lastIndexOf(":") + (s.substring(0, lio).trim, s.substring(lio + 1).trim) + }) keyVals.toMap } /** - * Parse a comma separated string into a sequence of strings. - * Whitespace surrounding the comma will be removed. + * Parse a comma separated string into a sequence of strings. Whitespace surrounding the comma will be removed. */ def parseCsvList(csvList: String): Seq[String] = { if (csvList == null || csvList.isEmpty) @@ -197,8 +203,10 @@ object CoreUtils { /** * Create a circular (looping) iterator over a collection. - * @param coll An iterable over the underlying collection. - * @return A circular iterator over the collection. + * @param coll + * An iterable over the underlying collection. + * @return + * A circular iterator over the collection. */ def circularIterator[T](coll: Iterable[T]) = for (_ <- Iterator.continually(1); t <- coll) yield t @@ -224,35 +232,49 @@ object CoreUtils { */ def duplicates[T](s: Iterable[T]): Iterable[T] = { s.groupBy(identity) - .map { case (k, l) => (k, l.size)} + .map { case (k, l) => (k, l.size) } .filter { case (_, l) => l > 1 } .keys } - def listenerListToEndPoints(listeners: String, securityProtocolMap: Map[ListenerName, SecurityProtocol]): Seq[EndPoint] = { + def listenerListToEndPoints( + listeners: String, + securityProtocolMap: Map[ListenerName, SecurityProtocol] + ): Seq[EndPoint] = { listenerListToEndPoints(listeners, securityProtocolMap, true) } - def listenerListToEndPoints(listeners: String, securityProtocolMap: Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[EndPoint] = { + def listenerListToEndPoints( + listeners: String, + securityProtocolMap: Map[ListenerName, SecurityProtocol], + requireDistinctPorts: Boolean + ): Seq[EndPoint] = { def validate(endPoints: Seq[EndPoint]): Unit = { // filter port 0 for unit tests val portsExcludingZero = endPoints.map(_.port).filter(_ != 0) val distinctListenerNames = endPoints.map(_.listenerName).distinct - require(distinctListenerNames.size == endPoints.size, s"Each listener must have a different name, listeners: $listeners") + require( + distinctListenerNames.size == endPoints.size, + s"Each listener must have a different name, listeners: $listeners" + ) if (requireDistinctPorts) { val distinctPorts = portsExcludingZero.distinct - require(distinctPorts.size == portsExcludingZero.size, s"Each listener must have a different port, listeners: $listeners") + require( + distinctPorts.size == portsExcludingZero.size, + s"Each listener must have a different port, listeners: $listeners" + ) } } - val endPoints = try { - val listenerList = parseCsvList(listeners) - listenerList.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) - } catch { - case e: Exception => - throw new IllegalArgumentException(s"Error creating broker listeners from '$listeners': ${e.getMessage}", e) - } + val endPoints = + try { + val listenerList = parseCsvList(listeners) + listenerList.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) + } catch { + case e: Exception => + throw new IllegalArgumentException(s"Error creating broker listeners from '$listeners': ${e.getMessage}", e) + } validate(endPoints) endPoints } @@ -281,10 +303,9 @@ object CoreUtils { } /** - * Atomic `getOrElseUpdate` for concurrent maps. This is optimized for the case where - * keys often exist in the map, avoiding the need to create a new value. `createValue` - * may be invoked more than once if multiple threads attempt to insert a key at the same - * time, but the same inserted value will be returned to all threads. + * Atomic `getOrElseUpdate` for concurrent maps. This is optimized for the case where keys often exist in the map, + * avoiding the need to create a new value. `createValue` may be invoked more than once if multiple threads attempt to + * insert a key at the same time, but the same inserted value will be returned to all threads. * * In Scala 2.12, `ConcurrentMap.getOrElse` has the same behaviour as this method, but JConcurrentMapWrapper that * wraps Java maps does not. diff --git a/app/src/main/scala/utils/Logging.scala b/app/src/main/scala/utils/Logging.scala index 0221821..7a7afc8 100644 --- a/app/src/main/scala/utils/Logging.scala +++ b/app/src/main/scala/utils/Logging.scala @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,13 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.utils import com.typesafe.scalalogging.Logger import org.slf4j.{LoggerFactory, Marker, MarkerFactory} - object Log4jControllerRegistration { private val logger = Logger(this.getClass.getName) @@ -53,7 +51,7 @@ trait Logging { def trace(msg: => String): Unit = logger.trace(msgWithLogIdent(msg)) - def trace(msg: => String, e: => Throwable): Unit = logger.trace(msgWithLogIdent(msg),e) + def trace(msg: => String, e: => Throwable): Unit = logger.trace(msgWithLogIdent(msg), e) def isDebugEnabled: Boolean = logger.underlying.isDebugEnabled @@ -61,19 +59,19 @@ trait Logging { def debug(msg: => String): Unit = logger.debug(msgWithLogIdent(msg)) - def debug(msg: => String, e: => Throwable): Unit = logger.debug(msgWithLogIdent(msg),e) + def debug(msg: => String, e: => Throwable): Unit = logger.debug(msgWithLogIdent(msg), e) def info(msg: => String): Unit = logger.info(msgWithLogIdent(msg)) - def info(msg: => String,e: => Throwable): Unit = logger.info(msgWithLogIdent(msg),e) + def info(msg: => String, e: => Throwable): Unit = logger.info(msgWithLogIdent(msg), e) def warn(msg: => String): Unit = logger.warn(msgWithLogIdent(msg)) - def warn(msg: => String, e: => Throwable): Unit = logger.warn(msgWithLogIdent(msg),e) + def warn(msg: => String, e: => Throwable): Unit = logger.warn(msgWithLogIdent(msg), e) def error(msg: => String): Unit = logger.error(msgWithLogIdent(msg)) - def error(msg: => String, e: => Throwable): Unit = logger.error(msgWithLogIdent(msg),e) + def error(msg: => String, e: => Throwable): Unit = logger.error(msgWithLogIdent(msg), e) def fatal(msg: => String): Unit = logger.error(Logging.FatalMarker, msgWithLogIdent(msg)) diff --git a/app/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala b/app/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala index e81c60c..9219d5c 100644 --- a/app/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala +++ b/app/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala @@ -1,12 +1,12 @@ /* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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, @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.admin import kafka.integration.KafkaServerTestHarness @@ -42,9 +41,11 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness { super.setUp(testInfo) createTopic(topicName, 1, 1.toShort) produceMessages() - adminClient = Admin.create(Map[String, Object]( - AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> bootstrapServers() - ).asJava) + adminClient = Admin.create( + Map[String, Object]( + AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> bootstrapServers() + ).asJava + ) } @AfterEach @@ -72,22 +73,25 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness { assertEquals(1, maxTimestampOffset.offset()) } - private def runFetchOffsets(adminClient: Admin, - offsetSpec: OffsetSpec): ListOffsetsResult.ListOffsetsResultInfo = { + private def runFetchOffsets(adminClient: Admin, offsetSpec: OffsetSpec): ListOffsetsResult.ListOffsetsResultInfo = { val tp = new TopicPartition(topicName, 0) - adminClient.listOffsets(Map( - tp -> offsetSpec - ).asJava, new ListOffsetsOptions()).all().get().get(tp) + adminClient + .listOffsets( + Map( + tp -> offsetSpec + ).asJava, + new ListOffsetsOptions() + ) + .all() + .get() + .get(tp) } def produceMessages(): Unit = { val records = Seq( - new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 100L, - null, new Array[Byte](10000)), - new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 999L, - null, new Array[Byte](10000)), - new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 200L, - null, new Array[Byte](10000)), + new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 100L, null, new Array[Byte](10000)), + new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 999L, null, new Array[Byte](10000)), + new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 200L, null, new Array[Byte](10000)) ) TestUtils.produceMessages(servers, records, -1) } diff --git a/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index eadb197..66eaf78 100644 --- a/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/app/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -67,7 +67,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { // TODO HSTREAM // properties.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown - properties.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + properties.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset // properties.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") // properties.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "100") // set small enough session timeout // properties.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, groupMaxSessionTimeoutMs.toString) diff --git a/app/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/app/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala index 2e0d323..6a4e2b4 100644 --- a/app/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala +++ b/app/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala @@ -1,202 +1,216 @@ - /* - * 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.api +/* + * 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.api - import org.junit.jupiter.api.Disabled +import org.junit.jupiter.api.Disabled - import java.util - import java.util.Properties - import java.util.concurrent.ExecutionException +import java.util +import java.util.Properties +import java.util.concurrent.ExecutionException // import kafka.security.authorizer.AclEntry - import kafka.server.KafkaConfig - import kafka.utils.Logging - import kafka.utils.TestUtils._ - import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription} - import org.apache.kafka.common.Uuid - import org.apache.kafka.common.acl.AclOperation - import org.apache.kafka.common.errors.{TopicExistsException, UnknownTopicOrPartitionException} - import org.apache.kafka.common.resource.ResourceType - import org.apache.kafka.common.utils.Utils - import org.junit.jupiter.api.Assertions._ - import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} +import kafka.server.KafkaConfig +import kafka.utils.Logging +import kafka.utils.TestUtils._ +import org.apache.kafka.clients.admin.{ + Admin, + AdminClientConfig, + CreateTopicsOptions, + CreateTopicsResult, + DescribeClusterOptions, + DescribeTopicsOptions, + NewTopic, + TopicDescription +} +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.acl.AclOperation +import org.apache.kafka.common.errors.{TopicExistsException, UnknownTopicOrPartitionException} +import org.apache.kafka.common.resource.ResourceType +import org.apache.kafka.common.utils.Utils +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} - import scala.jdk.CollectionConverters._ - import scala.collection.Seq - import scala.compat.java8.OptionConverters._ +import scala.jdk.CollectionConverters._ +import scala.collection.Seq +import scala.compat.java8.OptionConverters._ - /** - * Base integration test cases for [[Admin]]. Each test case added here will be executed - * in extending classes. Typically we prefer to write basic Admin functionality test cases in - * [[kafka.api.PlaintextAdminIntegrationTest]] rather than here to avoid unnecessary execution - * time to the build. However, if an admin API involves differing interactions with - * authentication/authorization layers, we may add the test case here. - */ - @Timeout(120) - abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logging { - def brokerCount = 3 +/** + * Base integration test cases for [[Admin]]. Each test case added here will be executed in extending classes. Typically + * we prefer to write basic Admin functionality test cases in [[kafka.api.PlaintextAdminIntegrationTest]] rather than + * here to avoid unnecessary execution time to the build. However, if an admin API involves differing interactions with + * authentication/authorization layers, we may add the test case here. + */ +@Timeout(120) +abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logging { + def brokerCount = 3 // override def logDirCount = 2 - var testInfo: TestInfo = _ + var testInfo: TestInfo = _ - var client: Admin = _ + var client: Admin = _ - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - this.testInfo = testInfo - super.setUp(testInfo) + @BeforeEach + override def setUp(testInfo: TestInfo): Unit = { + this.testInfo = testInfo + super.setUp(testInfo) // waitUntilBrokerMetadataIsPropagated(brokers) - } + } - @AfterEach - override def tearDown(): Unit = { - if (client != null) - Utils.closeQuietly(client, "AdminClient") - super.tearDown() - } + @AfterEach + override def tearDown(): Unit = { + if (client != null) + Utils.closeQuietly(client, "AdminClient") + super.tearDown() + } - @Test - @Disabled("Require CreateTopicRequest V4") - def testCreateDeleteTopics(): Unit = { - client = Admin.create(createConfig) - val topics = Seq("mytopic", "mytopic2", "mytopic3") - val newTopics = Seq( - new NewTopic("mytopic", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava), - new NewTopic("mytopic2", 3, 3.toShort), - new NewTopic("mytopic3", Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava) - ) - val validateResult = client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)) - validateResult.all.get() - waitForTopics(client, List(), topics) + @Test + @Disabled("Require CreateTopicRequest V4") + def testCreateDeleteTopics(): Unit = { + client = Admin.create(createConfig) + val topics = Seq("mytopic", "mytopic2", "mytopic3") + val newTopics = Seq( + new NewTopic( + "mytopic", + Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava + ), + new NewTopic("mytopic2", 3, 3.toShort), + new NewTopic("mytopic3", Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava) + ) + val validateResult = client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)) + validateResult.all.get() + waitForTopics(client, List(), topics) - def validateMetadataAndConfigs(result: CreateTopicsResult): Unit = { - assertEquals(2, result.numPartitions("mytopic").get()) - assertEquals(2, result.replicationFactor("mytopic").get()) - assertEquals(3, result.numPartitions("mytopic2").get()) - assertEquals(3, result.replicationFactor("mytopic2").get()) - assertEquals(configs.head.numPartitions, result.numPartitions("mytopic3").get()) - assertEquals(configs.head.defaultReplicationFactor, result.replicationFactor("mytopic3").get()) - assertFalse(result.config("mytopic").get().entries.isEmpty) - } - validateMetadataAndConfigs(validateResult) + def validateMetadataAndConfigs(result: CreateTopicsResult): Unit = { + assertEquals(2, result.numPartitions("mytopic").get()) + assertEquals(2, result.replicationFactor("mytopic").get()) + assertEquals(3, result.numPartitions("mytopic2").get()) + assertEquals(3, result.replicationFactor("mytopic2").get()) + assertEquals(configs.head.numPartitions, result.numPartitions("mytopic3").get()) + assertEquals(configs.head.defaultReplicationFactor, result.replicationFactor("mytopic3").get()) + assertFalse(result.config("mytopic").get().entries.isEmpty) + } + validateMetadataAndConfigs(validateResult) - val createResult = client.createTopics(newTopics.asJava) - createResult.all.get() - waitForTopics(client, topics, List()) - validateMetadataAndConfigs(createResult) - val topicIds = getTopicIds() - topics.foreach { topic => - assertNotEquals(Uuid.ZERO_UUID, createResult.topicId(topic).get()) - assertEquals(topicIds(topic), createResult.topicId(topic).get()) - } + val createResult = client.createTopics(newTopics.asJava) + createResult.all.get() + waitForTopics(client, topics, List()) + validateMetadataAndConfigs(createResult) + val topicIds = getTopicIds() + topics.foreach { topic => + assertNotEquals(Uuid.ZERO_UUID, createResult.topicId(topic).get()) + assertEquals(topicIds(topic), createResult.topicId(topic).get()) + } + val failedCreateResult = client.createTopics(newTopics.asJava) + val results = failedCreateResult.values() + assertTrue(results.containsKey("mytopic")) + assertFutureExceptionTypeEquals(results.get("mytopic"), classOf[TopicExistsException]) + assertTrue(results.containsKey("mytopic2")) + assertFutureExceptionTypeEquals(results.get("mytopic2"), classOf[TopicExistsException]) + assertTrue(results.containsKey("mytopic3")) + assertFutureExceptionTypeEquals(results.get("mytopic3"), classOf[TopicExistsException]) + assertFutureExceptionTypeEquals(failedCreateResult.numPartitions("mytopic3"), classOf[TopicExistsException]) + assertFutureExceptionTypeEquals(failedCreateResult.replicationFactor("mytopic3"), classOf[TopicExistsException]) + assertFutureExceptionTypeEquals(failedCreateResult.config("mytopic3"), classOf[TopicExistsException]) - val failedCreateResult = client.createTopics(newTopics.asJava) - val results = failedCreateResult.values() - assertTrue(results.containsKey("mytopic")) - assertFutureExceptionTypeEquals(results.get("mytopic"), classOf[TopicExistsException]) - assertTrue(results.containsKey("mytopic2")) - assertFutureExceptionTypeEquals(results.get("mytopic2"), classOf[TopicExistsException]) - assertTrue(results.containsKey("mytopic3")) - assertFutureExceptionTypeEquals(results.get("mytopic3"), classOf[TopicExistsException]) - assertFutureExceptionTypeEquals(failedCreateResult.numPartitions("mytopic3"), classOf[TopicExistsException]) - assertFutureExceptionTypeEquals(failedCreateResult.replicationFactor("mytopic3"), classOf[TopicExistsException]) - assertFutureExceptionTypeEquals(failedCreateResult.config("mytopic3"), classOf[TopicExistsException]) + val topicToDescription = client.describeTopics(topics.asJava).allTopicNames.get() + assertEquals(topics.toSet, topicToDescription.keySet.asScala) - val topicToDescription = client.describeTopics(topics.asJava).allTopicNames.get() - assertEquals(topics.toSet, topicToDescription.keySet.asScala) + val topic0 = topicToDescription.get("mytopic") + assertEquals(false, topic0.isInternal) + assertEquals("mytopic", topic0.name) + assertEquals(2, topic0.partitions.size) + val topic0Partition0 = topic0.partitions.get(0) + assertEquals(1, topic0Partition0.leader.id) + assertEquals(0, topic0Partition0.partition) + assertEquals(Seq(1, 2), topic0Partition0.isr.asScala.map(_.id)) + assertEquals(Seq(1, 2), topic0Partition0.replicas.asScala.map(_.id)) + val topic0Partition1 = topic0.partitions.get(1) + assertEquals(2, topic0Partition1.leader.id) + assertEquals(1, topic0Partition1.partition) + assertEquals(Seq(2, 0), topic0Partition1.isr.asScala.map(_.id)) + assertEquals(Seq(2, 0), topic0Partition1.replicas.asScala.map(_.id)) - val topic0 = topicToDescription.get("mytopic") - assertEquals(false, topic0.isInternal) - assertEquals("mytopic", topic0.name) - assertEquals(2, topic0.partitions.size) - val topic0Partition0 = topic0.partitions.get(0) - assertEquals(1, topic0Partition0.leader.id) - assertEquals(0, topic0Partition0.partition) - assertEquals(Seq(1, 2), topic0Partition0.isr.asScala.map(_.id)) - assertEquals(Seq(1, 2), topic0Partition0.replicas.asScala.map(_.id)) - val topic0Partition1 = topic0.partitions.get(1) - assertEquals(2, topic0Partition1.leader.id) - assertEquals(1, topic0Partition1.partition) - assertEquals(Seq(2, 0), topic0Partition1.isr.asScala.map(_.id)) - assertEquals(Seq(2, 0), topic0Partition1.replicas.asScala.map(_.id)) + val topic1 = topicToDescription.get("mytopic2") + assertEquals(false, topic1.isInternal) + assertEquals("mytopic2", topic1.name) + assertEquals(3, topic1.partitions.size) + for (partitionId <- 0 until 3) { + val partition = topic1.partitions.get(partitionId) + assertEquals(partitionId, partition.partition) + assertEquals(3, partition.replicas.size) + partition.replicas.forEach { replica => + assertTrue(replica.id >= 0) + assertTrue(replica.id < brokerCount) + } + assertEquals( + partition.replicas.size, + partition.replicas.asScala.map(_.id).distinct.size, + "No duplicate replica ids" + ) - val topic1 = topicToDescription.get("mytopic2") - assertEquals(false, topic1.isInternal) - assertEquals("mytopic2", topic1.name) - assertEquals(3, topic1.partitions.size) - for (partitionId <- 0 until 3) { - val partition = topic1.partitions.get(partitionId) - assertEquals(partitionId, partition.partition) - assertEquals(3, partition.replicas.size) - partition.replicas.forEach { replica => - assertTrue(replica.id >= 0) - assertTrue(replica.id < brokerCount) - } - assertEquals(partition.replicas.size, partition.replicas.asScala.map(_.id).distinct.size, "No duplicate replica ids") + assertEquals(3, partition.isr.size) + assertEquals(partition.replicas, partition.isr) + assertTrue(partition.replicas.contains(partition.leader)) + } - assertEquals(3, partition.isr.size) - assertEquals(partition.replicas, partition.isr) - assertTrue(partition.replicas.contains(partition.leader)) - } + val topic3 = topicToDescription.get("mytopic3") + assertEquals("mytopic3", topic3.name) + assertEquals(configs.head.numPartitions, topic3.partitions.size) + assertEquals(configs.head.defaultReplicationFactor, topic3.partitions.get(0).replicas().size()) - val topic3 = topicToDescription.get("mytopic3") - assertEquals("mytopic3", topic3.name) - assertEquals(configs.head.numPartitions, topic3.partitions.size) - assertEquals(configs.head.defaultReplicationFactor, topic3.partitions.get(0).replicas().size()) + client.deleteTopics(topics.asJava).all.get() + waitForTopics(client, List(), topics) + } - client.deleteTopics(topics.asJava).all.get() - waitForTopics(client, List(), topics) - } + @Test + def testAuthorizedOperations(): Unit = { + client = Admin.create(createConfig) - @Test - def testAuthorizedOperations(): Unit = { - client = Admin.create(createConfig) + // without includeAuthorizedOperations flag + var result = client.describeCluster + assertNull(result.authorizedOperations().get()) - // without includeAuthorizedOperations flag - var result = client.describeCluster - assertNull(result.authorizedOperations().get()) - - //with includeAuthorizedOperations flag - result = client.describeCluster(new DescribeClusterOptions().includeAuthorizedOperations(true)) + // with includeAuthorizedOperations flag + result = client.describeCluster(new DescribeClusterOptions().includeAuthorizedOperations(true)) // var expectedOperations = configuredClusterPermissions.asJava // assertEquals(expectedOperations, result.authorizedOperations().get()) - val topic = "mytopic" - val newTopics = Seq(new NewTopic(topic, 3, 3.toShort)) - client.createTopics(newTopics.asJava).all.get() - waitForTopics(client, expectedPresent = Seq(topic), expectedMissing = List()) + val topic = "mytopic" + val newTopics = Seq(new NewTopic(topic, 3, 3.toShort)) + client.createTopics(newTopics.asJava).all.get() + waitForTopics(client, expectedPresent = Seq(topic), expectedMissing = List()) - // without includeAuthorizedOperations flag - var topicResult = getTopicMetadata(client, topic) - assertNull(topicResult.authorizedOperations) + // without includeAuthorizedOperations flag + var topicResult = getTopicMetadata(client, topic) + assertNull(topicResult.authorizedOperations) // //with includeAuthorizedOperations flag // topicResult = getTopicMetadata(client, topic, new DescribeTopicsOptions().includeAuthorizedOperations(true)) // expectedOperations = AclEntry.supportedOperations(ResourceType.TOPIC).asJava // assertEquals(expectedOperations, topicResult.authorizedOperations) - } + } // def configuredClusterPermissions: Set[AclOperation] = // AclEntry.supportedOperations(ResourceType.CLUSTER) - override def modifyConfigs(configs: Seq[Properties]): Unit = { - super.modifyConfigs(configs) - // For testCreateTopicsReturnsConfigs, set some static broker configurations so that we can - // verify that they show up in the "configs" output of CreateTopics. + override def modifyConfigs(configs: Seq[Properties]): Unit = { + super.modifyConfigs(configs) + // For testCreateTopicsReturnsConfigs, set some static broker configurations so that we can + // verify that they show up in the "configs" output of CreateTopics. // if (testInfo.getTestMethod.toString.contains("testCreateTopicsReturnsConfigs")) { // configs.foreach(config => { // config.setProperty(KafkaConfig.LogRollTimeHoursProp, "2") @@ -214,7 +228,7 @@ // if (!config.containsKey(KafkaConfig.SslTruststorePasswordProp)) // config.setProperty(KafkaConfig.SslTruststorePasswordProp, "some.invalid.pass") // } - } + } // // override def kraftControllerConfigs(): Seq[Properties] = { // val controllerConfig = new Properties() @@ -223,39 +237,48 @@ // controllerConfigs // } - def createConfig: util.Map[String, Object] = { - val config = new util.HashMap[String, Object] - config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) - config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000") - val securityProps: util.Map[Object, Object] = - adminClientSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) - securityProps.forEach { (key, value) => config.put(key.asInstanceOf[String], value) } - config - } + def createConfig: util.Map[String, Object] = { + val config = new util.HashMap[String, Object] + config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) + config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000") + val securityProps: util.Map[Object, Object] = + adminClientSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) + securityProps.forEach { (key, value) => config.put(key.asInstanceOf[String], value) } + config + } - def waitForTopics(client: Admin, expectedPresent: Seq[String], expectedMissing: Seq[String]): Unit = { - waitUntilTrue(() => { - val topics = client.listTopics.names.get() - expectedPresent.forall(topicName => topics.contains(topicName)) && - expectedMissing.forall(topicName => !topics.contains(topicName)) - }, "timed out waiting for topics") - } + def waitForTopics(client: Admin, expectedPresent: Seq[String], expectedMissing: Seq[String]): Unit = { + waitUntilTrue( + () => { + val topics = client.listTopics.names.get() + expectedPresent.forall(topicName => topics.contains(topicName)) && + expectedMissing.forall(topicName => !topics.contains(topicName)) + }, + "timed out waiting for topics" + ) + } - def getTopicMetadata(client: Admin, - topic: String, - describeOptions: DescribeTopicsOptions = new DescribeTopicsOptions, - expectedNumPartitionsOpt: Option[Int] = None): TopicDescription = { - var result: TopicDescription = null - waitUntilTrue(() => { - val topicResult = client.describeTopics(Set(topic).asJava, describeOptions).topicNameValues().get(topic) - try { - result = topicResult.get - expectedNumPartitionsOpt.map(_ == result.partitions.size).getOrElse(true) - } catch { - case e: ExecutionException if e.getCause.isInstanceOf[UnknownTopicOrPartitionException] => false // metadata may not have propagated yet, so retry - } - }, s"Timed out waiting for metadata for $topic") - result - } + def getTopicMetadata( + client: Admin, + topic: String, + describeOptions: DescribeTopicsOptions = new DescribeTopicsOptions, + expectedNumPartitionsOpt: Option[Int] = None + ): TopicDescription = { + var result: TopicDescription = null + waitUntilTrue( + () => { + val topicResult = client.describeTopics(Set(topic).asJava, describeOptions).topicNameValues().get(topic) + try { + result = topicResult.get + expectedNumPartitionsOpt.map(_ == result.partitions.size).getOrElse(true) + } catch { + case e: ExecutionException if e.getCause.isInstanceOf[UnknownTopicOrPartitionException] => + false // metadata may not have propagated yet, so retry + } + }, + s"Timed out waiting for metadata for $topic" + ) + result + } - } +} diff --git a/app/src/test/scala/integration/kafka/api/BaseAsyncConsumerTest.scala b/app/src/test/scala/integration/kafka/api/BaseAsyncConsumerTest.scala index a0252ab..a170259 100644 --- a/app/src/test/scala/integration/kafka/api/BaseAsyncConsumerTest.scala +++ b/app/src/test/scala/integration/kafka/api/BaseAsyncConsumerTest.scala @@ -19,7 +19,6 @@ package kafka.api import kafka.utils.TestUtils.waitUntilTrue import org.junit.jupiter.api.Test - class BaseAsyncConsumerTest extends AbstractConsumerTest { @Test def testCommitAsync(): Unit = { @@ -30,9 +29,13 @@ class BaseAsyncConsumerTest extends AbstractConsumerTest { val cb = new CountConsumerCommitCallback sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) consumer.commitAsync(cb) - waitUntilTrue(() => { - cb.successCount == 1 - }, "wait until commit is completed successfully", 5000) + waitUntilTrue( + () => { + cb.successCount == 1 + }, + "wait until commit is completed successfully", + 5000 + ) } @Test diff --git a/app/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/app/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index d1034de..e04211b 100644 --- a/app/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/app/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -1,16 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.api import java.time.Duration diff --git a/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala b/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala index a6cdf52..929c7f6 100644 --- a/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala +++ b/app/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala @@ -1,113 +1,117 @@ - /** - * 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.api - - import kafka.server.KafkaConfig - import kafka.utils.{EmptyTestInfo, TestUtils} - import org.apache.kafka.clients.admin.NewTopic - import org.apache.kafka.clients.consumer.ConsumerConfig - import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} - import org.junit.jupiter.api.Assertions._ - import org.junit.jupiter.params.ParameterizedTest - import org.junit.jupiter.params.provider.{Arguments, MethodSource} - - import java.lang.{Boolean => JBoolean} - import java.time.Duration - import java.util - import java.util.Collections - - /** - * Tests behavior of specifying auto topic creation configuration for the consumer and broker - */ - class ConsumerTopicCreationTest { - - @ParameterizedTest - @MethodSource(Array("parameters")) - def testAutoTopicCreation(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean): Unit = { - val testCase = new ConsumerTopicCreationTest.TestCase(brokerAutoTopicCreationEnable, consumerAllowAutoCreateTopics) - val testInfo = new EmptyTestInfo() { - override def getDisplayName: String = - s"testAutoTopicCreation with args-${brokerAutoTopicCreationEnable.toString},${consumerAllowAutoCreateTopics.toString})" - } - testCase.setUp(testInfo) - try testCase.test() finally testCase.tearDown() - } - - } - - object ConsumerTopicCreationTest { - - private class TestCase(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean) extends IntegrationTestHarness { - private val topic_1 = "topic-1" - private val topic_2 = "topic-2" - private val producerClientId = "ConsumerTestProducer" - private val consumerClientId = "ConsumerTestConsumer" - - // configure server properties - // TODO: ENABLE_FOR_HSTREAM - // this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown - this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, brokerAutoTopicCreationEnable.toString) - - // configure client properties - this.producerConfig.setProperty(ProducerConfig.CLIENT_ID_CONFIG, producerClientId) - this.consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, consumerClientId) - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") - this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") - this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "100") - this.consumerConfig.setProperty(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, consumerAllowAutoCreateTopics.toString) - override protected def brokerCount: Int = 1 - - - def test(): Unit = { - val consumer = createConsumer() - val producer = createProducer() - val adminClient = createAdminClient() - val record = new ProducerRecord(topic_1, 0, "key".getBytes, "value".getBytes) - - // create `topic_1` and produce a record to it - adminClient.createTopics(Collections.singleton(new NewTopic(topic_1, 1, 1.toShort))).all.get - producer.send(record).get - - consumer.subscribe(util.Arrays.asList(topic_1, topic_2)) - - // Wait until the produced record was consumed. This guarantees that metadata request for `topic_2` was sent to the - // broker. - TestUtils.waitUntilTrue(() => { - consumer.poll(Duration.ofMillis(100)).count > 0 - }, "Timed out waiting to consume") - - // MetadataRequest is guaranteed to create the topic znode if creation was required - // KAFKA_TO_HSTREAM - // val topicCreated = zkClient.getAllTopicsInCluster().contains(topic_2) - val topicCreated = TestUtils.topicExists(createAdminClient(), topic_2) - if (brokerAutoTopicCreationEnable && consumerAllowAutoCreateTopics) - assertTrue(topicCreated) - else - assertFalse(topicCreated) - } - } - - def parameters: java.util.stream.Stream[Arguments] = { - val data = new java.util.ArrayList[Arguments]() - for (brokerAutoTopicCreationEnable <- Array(JBoolean.TRUE, JBoolean.FALSE)) - for (consumerAutoCreateTopicsPolicy <- Array(JBoolean.TRUE, JBoolean.FALSE)) - data.add(Arguments.of(brokerAutoTopicCreationEnable, consumerAutoCreateTopicsPolicy)) - data.stream() - } - } +/* + * 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.api + +import kafka.server.KafkaConfig +import kafka.utils.{EmptyTestInfo, TestUtils} +import org.apache.kafka.clients.admin.NewTopic +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{Arguments, MethodSource} + +import java.lang.{Boolean => JBoolean} +import java.time.Duration +import java.util +import java.util.Collections + +/** + * Tests behavior of specifying auto topic creation configuration for the consumer and broker + */ +class ConsumerTopicCreationTest { + + @ParameterizedTest + @MethodSource(Array("parameters")) + def testAutoTopicCreation(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean): Unit = { + val testCase = new ConsumerTopicCreationTest.TestCase(brokerAutoTopicCreationEnable, consumerAllowAutoCreateTopics) + val testInfo = new EmptyTestInfo() { + override def getDisplayName: String = + s"testAutoTopicCreation with args-${brokerAutoTopicCreationEnable.toString},${consumerAllowAutoCreateTopics.toString})" + } + testCase.setUp(testInfo) + try testCase.test() + finally testCase.tearDown() + } + +} + +object ConsumerTopicCreationTest { + + private class TestCase(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean) + extends IntegrationTestHarness { + private val topic_1 = "topic-1" + private val topic_2 = "topic-2" + private val producerClientId = "ConsumerTestProducer" + private val consumerClientId = "ConsumerTestConsumer" + + // configure server properties + // TODO: ENABLE_FOR_HSTREAM + // this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, brokerAutoTopicCreationEnable.toString) + + // configure client properties + this.producerConfig.setProperty(ProducerConfig.CLIENT_ID_CONFIG, producerClientId) + this.consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, consumerClientId) + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "100") + this.consumerConfig + .setProperty(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, consumerAllowAutoCreateTopics.toString) + override protected def brokerCount: Int = 1 + + def test(): Unit = { + val consumer = createConsumer() + val producer = createProducer() + val adminClient = createAdminClient() + val record = new ProducerRecord(topic_1, 0, "key".getBytes, "value".getBytes) + + // create `topic_1` and produce a record to it + adminClient.createTopics(Collections.singleton(new NewTopic(topic_1, 1, 1.toShort))).all.get + producer.send(record).get + + consumer.subscribe(util.Arrays.asList(topic_1, topic_2)) + + // Wait until the produced record was consumed. This guarantees that metadata request for `topic_2` was sent to the + // broker. + TestUtils.waitUntilTrue( + () => { + consumer.poll(Duration.ofMillis(100)).count > 0 + }, + "Timed out waiting to consume" + ) + + // MetadataRequest is guaranteed to create the topic znode if creation was required + // KAFKA_TO_HSTREAM + // val topicCreated = zkClient.getAllTopicsInCluster().contains(topic_2) + val topicCreated = TestUtils.topicExists(createAdminClient(), topic_2) + if (brokerAutoTopicCreationEnable && consumerAllowAutoCreateTopics) + assertTrue(topicCreated) + else + assertFalse(topicCreated) + } + } + + def parameters: java.util.stream.Stream[Arguments] = { + val data = new java.util.ArrayList[Arguments]() + for (brokerAutoTopicCreationEnable <- Array(JBoolean.TRUE, JBoolean.FALSE)) + for (consumerAutoCreateTopicsPolicy <- Array(JBoolean.TRUE, JBoolean.FALSE)) + data.add(Arguments.of(brokerAutoTopicCreationEnable, consumerAutoCreateTopicsPolicy)) + data.stream() + } +} diff --git a/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 8fe6d00..9cf1372 100644 --- a/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/app/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -1,185 +1,198 @@ - /** - * 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.api - - import java.io.File - import java.net.InetAddress - import java.lang.{Long => JLong} - import java.time.{Duration => JDuration} - import java.util.Arrays.asList - import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} - import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} - import java.util.{Collections, Optional, Properties} - import java.{time, util} - import kafka.integration.KafkaServerTestHarness - import kafka.server.Defaults - import kafka.utils.{TestInfoUtils, TestUtils} +/* + * 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.api + +import java.io.File +import java.net.InetAddress +import java.lang.{Long => JLong} +import java.time.{Duration => JDuration} +import java.util.Arrays.asList +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} +import java.util.{Collections, Optional, Properties} +import java.{time, util} +import kafka.integration.KafkaServerTestHarness +import kafka.server.Defaults +import kafka.utils.{TestInfoUtils, TestUtils} // import kafka.security.authorizer.AclEntry // import kafka.server.metadata.KRaftMetadataCache // import kafka.server.{Defaults, DynamicConfig, KafkaConfig} - import kafka.utils.TestUtils._ +import kafka.utils.TestUtils._ // import kafka.utils.{Log4jController, TestInfoUtils, TestUtils} - import org.apache.kafka.clients.HostResolver - import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource - import org.apache.kafka.clients.admin._ - import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} - import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} - import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} - import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig} - import org.apache.kafka.common.errors._ - import org.apache.kafka.common.requests.{DeleteRecordsRequest, MetadataResponse} - import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} - import org.apache.kafka.common.utils.{Time, Utils} - import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid} +import org.apache.kafka.clients.HostResolver +import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource +import org.apache.kafka.clients.admin._ +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} +import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig} +import org.apache.kafka.common.errors._ +import org.apache.kafka.common.requests.{DeleteRecordsRequest, MetadataResponse} +import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} +import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.common.{ + ConsumerGroupState, + ElectionType, + TopicCollection, + TopicPartition, + TopicPartitionInfo, + TopicPartitionReplica, + Uuid +} // import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT // import org.apache.kafka.storage.internals.log.LogConfig - import org.junit.jupiter.api.Assertions._ - import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo} - import org.junit.jupiter.params.ParameterizedTest - import org.junit.jupiter.params.provider.ValueSource - import org.slf4j.LoggerFactory - - import java.util.AbstractMap.SimpleImmutableEntry - import scala.annotation.nowarn - import scala.collection.Seq - import scala.compat.java8.OptionConverters._ - import scala.concurrent.duration.Duration - import scala.concurrent.{Await, Future} - import scala.jdk.CollectionConverters._ - import scala.util.Random - - /** - * An integration test of the KafkaAdminClient. - * - * Also see [[org.apache.kafka.clients.admin.KafkaAdminClientTest]] for unit tests of the admin client. - */ - class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource +import org.slf4j.LoggerFactory + +import java.util.AbstractMap.SimpleImmutableEntry +import scala.annotation.nowarn +import scala.collection.Seq +import scala.compat.java8.OptionConverters._ +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future} +import scala.jdk.CollectionConverters._ +import scala.util.Random + +/** + * An integration test of the KafkaAdminClient. + * + * Also see [[org.apache.kafka.clients.admin.KafkaAdminClientTest]] for unit tests of the admin client. + */ +class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // import PlaintextAdminIntegrationTest._ - val topic = "topic" - val partition = 0 - val topicPartition = new TopicPartition(topic, partition) + val topic = "topic" + val partition = 0 + val topicPartition = new TopicPartition(topic, partition) - private var brokerLoggerConfigResource: ConfigResource = _ - private val changedBrokerLoggers = scala.collection.mutable.Set[String]() + private var brokerLoggerConfigResource: ConfigResource = _ + private val changedBrokerLoggers = scala.collection.mutable.Set[String]() - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - super.setUp(testInfo) - brokerLoggerConfigResource = new ConfigResource( - ConfigResource.Type.BROKER_LOGGER, brokers.head.config.brokerId.toString) - } + @BeforeEach + override def setUp(testInfo: TestInfo): Unit = { + super.setUp(testInfo) + brokerLoggerConfigResource = + new ConfigResource(ConfigResource.Type.BROKER_LOGGER, brokers.head.config.brokerId.toString) + } - @AfterEach - override def tearDown(): Unit = { + @AfterEach + override def tearDown(): Unit = { // teardownBrokerLoggers() - super.tearDown() - } - - @Test - def testClose(): Unit = { - val client = Admin.create(createConfig) - client.close() - client.close() // double close has no effect - } - - @Test - def testListNodes(): Unit = { - client = Admin.create(createConfig) - val brokerStrs = bootstrapServers().split(",").toList.sorted - var nodeStrs: List[String] = null - do { - val nodes = client.describeCluster().nodes().get().asScala - nodeStrs = nodes.map ( node => s"${node.host}:${node.port}" ).toList.sorted - } while (nodeStrs.size < brokerStrs.size) - assertEquals(brokerStrs.mkString(","), nodeStrs.mkString(",")) - } - - @Test - def testAdminClientHandlingBadIPWithoutTimeout(): Unit = { - val config = createConfig - config.put(AdminClientConfig.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, "1000") - val returnBadAddressFirst = new HostResolver { - override def resolve(host: String): Array[InetAddress] = { - Array[InetAddress](InetAddress.getByName("10.200.20.100"), InetAddress.getByName(host)) - } - } - client = AdminClientTestUtils.create(config, returnBadAddressFirst) - // simply check that a call, e.g. describeCluster, returns normally - client.describeCluster().nodes().get() - } - - @Test - def testCreateExistingTopicsThrowTopicExistsException(): Unit = { - client = Admin.create(createConfig) - val topic = "mytopic" - val topics = Seq(topic) - val newTopics = Seq(new NewTopic(topic, 1, 1.toShort)) - - client.createTopics(newTopics.asJava).all.get() - waitForTopics(client, topics, List()) - - val newTopicsWithInvalidRF = Seq(new NewTopic(topic, 1, (brokers.size + 1).toShort)) - val e = assertThrows(classOf[ExecutionException], - () => client.createTopics(newTopicsWithInvalidRF.asJava, new CreateTopicsOptions().validateOnly(true)).all.get()) - assertTrue(e.getCause.isInstanceOf[TopicExistsException]) - } - - @Test - @Disabled("Require CreateTopicsRequest >= V4") - def testDeleteTopicsWithIds(): Unit = { - client = Admin.create(createConfig) - val topics = Seq("mytopic", "mytopic2", "mytopic3") - val newTopics = Seq( - new NewTopic("mytopic", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava), - new NewTopic("mytopic2", 3, 3.toShort), - new NewTopic("mytopic3", Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava) - ) - val createResult = client.createTopics(newTopics.asJava) - createResult.all.get() - waitForTopics(client, topics, List()) - val topicIds = getTopicIds().values.toSet - - client.deleteTopics(TopicCollection.ofTopicIds(topicIds.asJava)).all.get() - waitForTopics(client, List(), topics) - } - - /** - * describe should not auto create topics - */ - @Test - // TODO: 检查结果 - def testDescribeNonExistingTopic(): Unit = { - client = Admin.create(createConfig) - - val existingTopic = "existing-topic" - client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get() - waitForTopics(client, Seq(existingTopic), List()) - - val nonExistingTopic = "non-existing" - val results = client.describeTopics(Seq(nonExistingTopic, existingTopic).asJava).topicNameValues() - assertEquals(existingTopic, results.get(existingTopic).get.name) - assertFutureExceptionTypeEquals(results.get(nonExistingTopic), classOf[UnknownTopicOrPartitionException]) + super.tearDown() + } + + @Test + def testClose(): Unit = { + val client = Admin.create(createConfig) + client.close() + client.close() // double close has no effect + } + + @Test + def testListNodes(): Unit = { + client = Admin.create(createConfig) + val brokerStrs = bootstrapServers().split(",").toList.sorted + var nodeStrs: List[String] = null + do { + val nodes = client.describeCluster().nodes().get().asScala + nodeStrs = nodes.map(node => s"${node.host}:${node.port}").toList.sorted + } while (nodeStrs.size < brokerStrs.size) + assertEquals(brokerStrs.mkString(","), nodeStrs.mkString(",")) + } + + @Test + def testAdminClientHandlingBadIPWithoutTimeout(): Unit = { + val config = createConfig + config.put(AdminClientConfig.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, "1000") + val returnBadAddressFirst = new HostResolver { + override def resolve(host: String): Array[InetAddress] = { + Array[InetAddress](InetAddress.getByName("10.200.20.100"), InetAddress.getByName(host)) + } + } + client = AdminClientTestUtils.create(config, returnBadAddressFirst) + // simply check that a call, e.g. describeCluster, returns normally + client.describeCluster().nodes().get() + } + + @Test + def testCreateExistingTopicsThrowTopicExistsException(): Unit = { + client = Admin.create(createConfig) + val topic = "mytopic" + val topics = Seq(topic) + val newTopics = Seq(new NewTopic(topic, 1, 1.toShort)) + + client.createTopics(newTopics.asJava).all.get() + waitForTopics(client, topics, List()) + + val newTopicsWithInvalidRF = Seq(new NewTopic(topic, 1, (brokers.size + 1).toShort)) + val e = assertThrows( + classOf[ExecutionException], + () => client.createTopics(newTopicsWithInvalidRF.asJava, new CreateTopicsOptions().validateOnly(true)).all.get() + ) + assertTrue(e.getCause.isInstanceOf[TopicExistsException]) + } + + @Test + @Disabled("Require CreateTopicsRequest >= V4") + def testDeleteTopicsWithIds(): Unit = { + client = Admin.create(createConfig) + val topics = Seq("mytopic", "mytopic2", "mytopic3") + val newTopics = Seq( + new NewTopic( + "mytopic", + Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava + ), + new NewTopic("mytopic2", 3, 3.toShort), + new NewTopic("mytopic3", Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava) + ) + val createResult = client.createTopics(newTopics.asJava) + createResult.all.get() + waitForTopics(client, topics, List()) + val topicIds = getTopicIds().values.toSet + + client.deleteTopics(TopicCollection.ofTopicIds(topicIds.asJava)).all.get() + waitForTopics(client, List(), topics) + } + + /** + * describe should not auto create topics + */ + @Test + // TODO: 检查结果 + def testDescribeNonExistingTopic(): Unit = { + client = Admin.create(createConfig) + + val existingTopic = "existing-topic" + client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get() + waitForTopics(client, Seq(existingTopic), List()) + + val nonExistingTopic = "non-existing" + val results = client.describeTopics(Seq(nonExistingTopic, existingTopic).asJava).topicNameValues() + assertEquals(existingTopic, results.get(existingTopic).get.name) + assertFutureExceptionTypeEquals(results.get(nonExistingTopic), classOf[UnknownTopicOrPartitionException]) // if (!isKRaftTest()) { // assertEquals(None, zkClient.getTopicPartitionCount(nonExistingTopic)) // } - } + } - //TODO:KAFKA_TO_HSTREAM: HStream doesn't support get metadataCache + // TODO:KAFKA_TO_HSTREAM: HStream doesn't support get metadataCache // @Test // def testDescribeTopicsWithIds(): Unit = { // client = Admin.create(createConfig) @@ -198,16 +211,16 @@ // assertThrows(classOf[ExecutionException], () => results.get(nonExistingTopicId).get).getCause.isInstanceOf[UnknownTopicIdException] // } // - @Test - @Disabled("Require support DescribeCluster Request") - def testDescribeCluster(): Unit = { - client = Admin.create(createConfig) - val result = client.describeCluster - info(s"result: ${result.nodes().get()}") - val nodes = result.nodes.get() - val clusterId = result.clusterId().get() + @Test + @Disabled("Require support DescribeCluster Request") + def testDescribeCluster(): Unit = { + client = Admin.create(createConfig) + val result = client.describeCluster + info(s"result: ${result.nodes().get()}") + val nodes = result.nodes.get() + val clusterId = result.clusterId().get() // assertEquals(brokers.head.dataPlaneRequestProcessor.clusterId, clusterId) - val controller = result.controller().get() + val controller = result.controller().get() // if (isKRaftTest()) { // // In KRaft, we return a random brokerId as the current controller. @@ -218,16 +231,16 @@ // getOrElse(MetadataResponse.NO_CONTROLLER_ID), controller.id) // } - val brokerIds = brokers.map(_.config.brokerId).toSet - assertTrue(brokerIds.contains(controller.id)) + val brokerIds = brokers.map(_.config.brokerId).toSet + assertTrue(brokerIds.contains(controller.id)) - val brokerEndpoints = bootstrapServers().split(",") - assertEquals(brokerEndpoints.size, nodes.size) - for (node <- nodes.asScala) { - val hostStr = s"${node.host}:${node.port}" - assertTrue(brokerEndpoints.contains(hostStr), s"Unknown host:port pair $hostStr in brokerVersionInfos") - } - } + val brokerEndpoints = bootstrapServers().split(",") + assertEquals(brokerEndpoints.size, nodes.size) + for (node <- nodes.asScala) { + val hostStr = s"${node.host}:${node.port}" + assertTrue(brokerEndpoints.contains(hostStr), s"Unknown host:port pair $hostStr in brokerVersionInfos") + } + } // @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) // @ValueSource(strings = Array("zk", "kraft")) @@ -453,45 +466,56 @@ // checkValidAlterConfigs(client, this, topicResource1, topicResource2) // } // - @Test - def testCreatePartitions(): Unit = { - client = Admin.create(createConfig) - - // Create topics - val topic1 = "create-partitions-topic-1" - createTopic(topic1) - - val topic2 = "create-partitions-topic-2" - createTopic(topic2, replicationFactor = 2) - - // assert that both the topics have 1 partition - val topic1_metadata = getTopicMetadata(client, topic1) - val topic2_metadata = getTopicMetadata(client, topic2) - assertEquals(1, topic1_metadata.partitions.size) - assertEquals(1, topic2_metadata.partitions.size) - - val validateOnly = new CreatePartitionsOptions().validateOnly(true) - val actuallyDoIt = new CreatePartitionsOptions().validateOnly(false) - - def partitions(topic: String, expectedNumPartitionsOpt: Option[Int]): util.List[TopicPartitionInfo] = { - getTopicMetadata(client, topic, expectedNumPartitionsOpt = expectedNumPartitionsOpt).partitions - } - - def numPartitions(topic: String, expectedNumPartitionsOpt: Option[Int] = None): Int = partitions(topic, expectedNumPartitionsOpt).size - - // validateOnly: try creating a new partition (no assignments), to bring the total to 3 partitions - var alterResult = client.createPartitions(Map(topic1 -> - NewPartitions.increaseTo(3)).asJava, validateOnly) - var altered = alterResult.values.get(topic1).get - assertEquals(1, numPartitions(topic1)) - - // try creating a new partition (no assignments), to bring the total to 3 partitions - alterResult = client.createPartitions(Map(topic1 -> - NewPartitions.increaseTo(3)).asJava, actuallyDoIt) - altered = alterResult.values.get(topic1).get - TestUtils.waitUntilTrue(() => numPartitions(topic1) == 3, "Timed out waiting for new partitions to appear") - - // KAFKA_TO_HSTREAM: HStream doesn't support partition assignment + @Test + def testCreatePartitions(): Unit = { + client = Admin.create(createConfig) + + // Create topics + val topic1 = "create-partitions-topic-1" + createTopic(topic1) + + val topic2 = "create-partitions-topic-2" + createTopic(topic2, replicationFactor = 2) + + // assert that both the topics have 1 partition + val topic1_metadata = getTopicMetadata(client, topic1) + val topic2_metadata = getTopicMetadata(client, topic2) + assertEquals(1, topic1_metadata.partitions.size) + assertEquals(1, topic2_metadata.partitions.size) + + val validateOnly = new CreatePartitionsOptions().validateOnly(true) + val actuallyDoIt = new CreatePartitionsOptions().validateOnly(false) + + def partitions(topic: String, expectedNumPartitionsOpt: Option[Int]): util.List[TopicPartitionInfo] = { + getTopicMetadata(client, topic, expectedNumPartitionsOpt = expectedNumPartitionsOpt).partitions + } + + def numPartitions(topic: String, expectedNumPartitionsOpt: Option[Int] = None): Int = + partitions(topic, expectedNumPartitionsOpt).size + + // validateOnly: try creating a new partition (no assignments), to bring the total to 3 partitions + var alterResult = client.createPartitions( + Map( + topic1 -> + NewPartitions.increaseTo(3) + ).asJava, + validateOnly + ) + var altered = alterResult.values.get(topic1).get + assertEquals(1, numPartitions(topic1)) + + // try creating a new partition (no assignments), to bring the total to 3 partitions + alterResult = client.createPartitions( + Map( + topic1 -> + NewPartitions.increaseTo(3) + ).asJava, + actuallyDoIt + ) + altered = alterResult.values.get(topic1).get + TestUtils.waitUntilTrue(() => numPartitions(topic1) == 3, "Timed out waiting for new partitions to appear") + + // KAFKA_TO_HSTREAM: HStream doesn't support partition assignment // // validateOnly: now try creating a new partition (with assignments), to bring the total to 3 partitions // val newPartition2Assignments = asList[util.List[Integer]](asList(0, 1), asList(1, 2)) // alterResult = client.createPartitions(Map(topic2 -> @@ -508,44 +532,60 @@ // assertEquals(Seq(0, 1), actualPartitions2.get(1).replicas.asScala.map(_.id).toList) // assertEquals(Seq(1, 2), actualPartitions2.get(2).replicas.asScala.map(_.id).toList) - alterResult = client.createPartitions(Map(topic2 -> NewPartitions.increaseTo(3)).asJava, actuallyDoIt) - altered = alterResult.values.get(topic2).get - TestUtils.waitUntilTrue(() => numPartitions(topic2) == 3, "Timed out waiting for new partitions to appear") - - // loop over error cases calling with+without validate-only - for (option <- Seq(validateOnly, actuallyDoIt)) { - val desc = if (option.validateOnly()) "validateOnly" else "validateOnly=false" - - // try a newCount which would be a decrease - alterResult = client.createPartitions(Map(topic1 -> - NewPartitions.increaseTo(1)).asJava, option) - - var e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(topic1).get, - () => s"$desc: Expect InvalidPartitionsException when newCount is a decrease") - assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException], desc) - var exceptionMsgStr = if (isKRaftTest()) { - "The topic create-partitions-topic-1 currently has 3 partition(s); 1 would not be an increase." - } else { - "Topic currently has 3 partitions, which is higher than the requested 1." - } - assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) - assertEquals(3, numPartitions(topic1), desc) - - // try a newCount which would be a noop (without assignment) - alterResult = client.createPartitions(Map(topic2 -> - NewPartitions.increaseTo(3)).asJava, option) - e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(topic2).get, - () => s"$desc: Expect InvalidPartitionsException when requesting a noop") - assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException], desc) - exceptionMsgStr = if (isKRaftTest()) { - "Topic already has 3 partition(s)." - } else { - "Topic already has 3 partitions." - } - assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) - assertEquals(3, numPartitions(topic2, Some(3)), desc) - - // KAFKA_TO_HSTREAM: HStream doesn't support partition assignment + alterResult = client.createPartitions(Map(topic2 -> NewPartitions.increaseTo(3)).asJava, actuallyDoIt) + altered = alterResult.values.get(topic2).get + TestUtils.waitUntilTrue(() => numPartitions(topic2) == 3, "Timed out waiting for new partitions to appear") + + // loop over error cases calling with+without validate-only + for (option <- Seq(validateOnly, actuallyDoIt)) { + val desc = if (option.validateOnly()) "validateOnly" else "validateOnly=false" + + // try a newCount which would be a decrease + alterResult = client.createPartitions( + Map( + topic1 -> + NewPartitions.increaseTo(1) + ).asJava, + option + ) + + var e = assertThrows( + classOf[ExecutionException], + () => alterResult.values.get(topic1).get, + () => s"$desc: Expect InvalidPartitionsException when newCount is a decrease" + ) + assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException], desc) + var exceptionMsgStr = if (isKRaftTest()) { + "The topic create-partitions-topic-1 currently has 3 partition(s); 1 would not be an increase." + } else { + "Topic currently has 3 partitions, which is higher than the requested 1." + } + assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) + assertEquals(3, numPartitions(topic1), desc) + + // try a newCount which would be a noop (without assignment) + alterResult = client.createPartitions( + Map( + topic2 -> + NewPartitions.increaseTo(3) + ).asJava, + option + ) + e = assertThrows( + classOf[ExecutionException], + () => alterResult.values.get(topic2).get, + () => s"$desc: Expect InvalidPartitionsException when requesting a noop" + ) + assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException], desc) + exceptionMsgStr = if (isKRaftTest()) { + "Topic already has 3 partition(s)." + } else { + "Topic already has 3 partitions." + } + assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) + assertEquals(3, numPartitions(topic2, Some(3)), desc) + + // KAFKA_TO_HSTREAM: HStream doesn't support partition assignment // // try a newCount which would be a noop (where the assignment matches current state) // alterResult = client.createPartitions(Map(topic2 -> // NewPartitions.increaseTo(3, newPartition2Assignments)).asJava, option) @@ -562,36 +602,51 @@ // assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) // assertEquals(3, numPartitions(topic2, Some(3)), desc) - // try a bad topic name - val unknownTopic = "an-unknown-topic" - alterResult = client.createPartitions(Map(unknownTopic -> - NewPartitions.increaseTo(2)).asJava, option) - e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(unknownTopic).get, - () => s"$desc: Expect InvalidTopicException when using an unknown topic") - assertTrue(e.getCause.isInstanceOf[UnknownTopicOrPartitionException], desc) - exceptionMsgStr = if (isKRaftTest()) { - "This server does not host this topic-partition." - } else { - "The topic 'an-unknown-topic' does not exist." - } - assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) - - // try an invalid newCount - alterResult = client.createPartitions(Map(topic1 -> - NewPartitions.increaseTo(-22)).asJava, option) - e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(topic1).get, - () => s"$desc: Expect InvalidPartitionsException when newCount is invalid") - assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException], desc) - exceptionMsgStr = if (isKRaftTest()) { - "The topic create-partitions-topic-1 currently has 3 partition(s); -22 would not be an increase." - } else { - "Topic currently has 3 partitions, which is higher than the requested -22." - } - assertEquals(exceptionMsgStr, e.getCause.getMessage, - desc) - assertEquals(3, numPartitions(topic1), desc) - - // KAFKA_TO_HSTREAM: HStream doesn't support partition assignment + // try a bad topic name + val unknownTopic = "an-unknown-topic" + alterResult = client.createPartitions( + Map( + unknownTopic -> + NewPartitions.increaseTo(2) + ).asJava, + option + ) + e = assertThrows( + classOf[ExecutionException], + () => alterResult.values.get(unknownTopic).get, + () => s"$desc: Expect InvalidTopicException when using an unknown topic" + ) + assertTrue(e.getCause.isInstanceOf[UnknownTopicOrPartitionException], desc) + exceptionMsgStr = if (isKRaftTest()) { + "This server does not host this topic-partition." + } else { + "The topic 'an-unknown-topic' does not exist." + } + assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) + + // try an invalid newCount + alterResult = client.createPartitions( + Map( + topic1 -> + NewPartitions.increaseTo(-22) + ).asJava, + option + ) + e = assertThrows( + classOf[ExecutionException], + () => alterResult.values.get(topic1).get, + () => s"$desc: Expect InvalidPartitionsException when newCount is invalid" + ) + assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException], desc) + exceptionMsgStr = if (isKRaftTest()) { + "The topic create-partitions-topic-1 currently has 3 partition(s); -22 would not be an increase." + } else { + "Topic currently has 3 partitions, which is higher than the requested -22." + } + assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) + assertEquals(3, numPartitions(topic1), desc) + + // KAFKA_TO_HSTREAM: HStream doesn't support partition assignment // // try assignments where the number of brokers != replication factor // alterResult = client.createPartitions(Map(topic1 -> // NewPartitions.increaseTo(4, asList(asList(1, 2)))).asJava, option) @@ -693,36 +748,45 @@ // } // assertEquals(exceptionMsgStr, e.getCause.getMessage, desc) // assertEquals(3, numPartitions(topic1), desc) - } - - // a mixed success, failure response - alterResult = client.createPartitions(Map( - topic1 -> NewPartitions.increaseTo(4), - topic2 -> NewPartitions.increaseTo(2)).asJava, actuallyDoIt) - // assert that the topic1 now has 4 partitions - altered = alterResult.values.get(topic1).get - TestUtils.waitUntilTrue(() => numPartitions(topic1) == 4, "Timed out waiting for new partitions to appear") - var e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(topic2).get) - assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException]) - val exceptionMsgStr = if (isKRaftTest()) { - "The topic create-partitions-topic-2 currently has 3 partition(s); 2 would not be an increase." - } else { - "Topic currently has 3 partitions, which is higher than the requested 2." - } - assertEquals(exceptionMsgStr, e.getCause.getMessage) - assertEquals(3, numPartitions(topic2)) - - // Delete the topic. Verify addition of partitions to deleted topic is not possible. In - // Zookeeper mode, the topic is queued for deletion. In KRaft, the deletion occurs - // immediately and hence we have a different Exception thrown in the response. - val deleteResult = client.deleteTopics(asList(topic1)) - deleteResult.topicNameValues.get(topic1).get - alterResult = client.createPartitions(Map(topic1 -> - NewPartitions.increaseTo(4)).asJava, validateOnly) - e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(topic1).get, - () => "Expect InvalidTopicException or UnknownTopicOrPartitionException when the topic is queued for deletion") - - // KAFKA_TO_HSTREAM + } + + // a mixed success, failure response + alterResult = client.createPartitions( + Map(topic1 -> NewPartitions.increaseTo(4), topic2 -> NewPartitions.increaseTo(2)).asJava, + actuallyDoIt + ) + // assert that the topic1 now has 4 partitions + altered = alterResult.values.get(topic1).get + TestUtils.waitUntilTrue(() => numPartitions(topic1) == 4, "Timed out waiting for new partitions to appear") + var e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(topic2).get) + assertTrue(e.getCause.isInstanceOf[InvalidPartitionsException]) + val exceptionMsgStr = if (isKRaftTest()) { + "The topic create-partitions-topic-2 currently has 3 partition(s); 2 would not be an increase." + } else { + "Topic currently has 3 partitions, which is higher than the requested 2." + } + assertEquals(exceptionMsgStr, e.getCause.getMessage) + assertEquals(3, numPartitions(topic2)) + + // Delete the topic. Verify addition of partitions to deleted topic is not possible. In + // Zookeeper mode, the topic is queued for deletion. In KRaft, the deletion occurs + // immediately and hence we have a different Exception thrown in the response. + val deleteResult = client.deleteTopics(asList(topic1)) + deleteResult.topicNameValues.get(topic1).get + alterResult = client.createPartitions( + Map( + topic1 -> + NewPartitions.increaseTo(4) + ).asJava, + validateOnly + ) + e = assertThrows( + classOf[ExecutionException], + () => alterResult.values.get(topic1).get, + () => "Expect InvalidTopicException or UnknownTopicOrPartitionException when the topic is queued for deletion" + ) + + // KAFKA_TO_HSTREAM // if (isKRaftTest()) { // assertTrue(e.getCause.isInstanceOf[UnknownTopicOrPartitionException], e.toString) // assertEquals("This server does not host this topic-partition.", e.getCause.getMessage) @@ -730,11 +794,11 @@ // assertTrue(e.getCause.isInstanceOf[InvalidTopicException], e.toString) // assertEquals("The topic is queued for deletion.", e.getCause.getMessage) // } - info(s"Got expected exception: ${e.getCause}") - assertTrue(e.getCause.isInstanceOf[UnknownTopicOrPartitionException], e.toString) - // KAFKA_TO_HSTREAM + info(s"Got expected exception: ${e.getCause}") + assertTrue(e.getCause.isInstanceOf[UnknownTopicOrPartitionException], e.toString) + // KAFKA_TO_HSTREAM // assertEquals("This server does not host this topic-partition.", e.getCause.getMessage) - } + } // TODO:ENABLE_FOR_HSTREAM, need support deleteRecords request // @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @@ -983,43 +1047,51 @@ // assertEquals(classOf[LeaderNotAvailableException], cause.getClass) // } - @Test - def testDescribeConfigsForTopic(): Unit = { - createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) - - val existingTopic = new ConfigResource(ConfigResource.Type.TOPIC, topic) - client.describeConfigs(Collections.singletonList(existingTopic)).values.get(existingTopic).get() - - val nonExistentTopic = new ConfigResource(ConfigResource.Type.TOPIC, "unknown") - val describeResult1 = client.describeConfigs(Collections.singletonList(nonExistentTopic)) - - assertTrue(assertThrows(classOf[ExecutionException], () => describeResult1.values.get(nonExistentTopic).get).getCause.isInstanceOf[UnknownTopicOrPartitionException]) - - val invalidTopic = new ConfigResource(ConfigResource.Type.TOPIC, "(invalid topic)") - val describeResult2 = client.describeConfigs(Collections.singletonList(invalidTopic)) - - assertTrue(assertThrows(classOf[ExecutionException], () => describeResult2.values.get(invalidTopic).get).getCause.isInstanceOf[InvalidTopicException]) - } - - private def subscribeAndWaitForAssignment(topic: String, consumer: KafkaConsumer[Array[Byte], Array[Byte]]): Unit = { - consumer.subscribe(Collections.singletonList(topic)) - TestUtils.pollUntilTrue(consumer, () => !consumer.assignment.isEmpty, "Expected non-empty assignment") - } - - private def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], - numRecords: Int, - topicPartition: TopicPartition): Unit = { - val futures = (0 until numRecords).map( i => { - val record = new ProducerRecord(topicPartition.topic, topicPartition.partition, s"$i".getBytes, s"$i".getBytes) - debug(s"Sending this record: $record") - producer.send(record) - }) - - futures.foreach(_.get) - } - - // TODO: ENABLE_FOR_HSTREAM: need to support alter config + @Test + def testDescribeConfigsForTopic(): Unit = { + createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) + client = Admin.create(createConfig) + + val existingTopic = new ConfigResource(ConfigResource.Type.TOPIC, topic) + client.describeConfigs(Collections.singletonList(existingTopic)).values.get(existingTopic).get() + + val nonExistentTopic = new ConfigResource(ConfigResource.Type.TOPIC, "unknown") + val describeResult1 = client.describeConfigs(Collections.singletonList(nonExistentTopic)) + + assertTrue( + assertThrows(classOf[ExecutionException], () => describeResult1.values.get(nonExistentTopic).get).getCause + .isInstanceOf[UnknownTopicOrPartitionException] + ) + + val invalidTopic = new ConfigResource(ConfigResource.Type.TOPIC, "(invalid topic)") + val describeResult2 = client.describeConfigs(Collections.singletonList(invalidTopic)) + + assertTrue( + assertThrows(classOf[ExecutionException], () => describeResult2.values.get(invalidTopic).get).getCause + .isInstanceOf[InvalidTopicException] + ) + } + + private def subscribeAndWaitForAssignment(topic: String, consumer: KafkaConsumer[Array[Byte], Array[Byte]]): Unit = { + consumer.subscribe(Collections.singletonList(topic)) + TestUtils.pollUntilTrue(consumer, () => !consumer.assignment.isEmpty, "Expected non-empty assignment") + } + + private def sendRecords( + producer: KafkaProducer[Array[Byte], Array[Byte]], + numRecords: Int, + topicPartition: TopicPartition + ): Unit = { + val futures = (0 until numRecords).map(i => { + val record = new ProducerRecord(topicPartition.topic, topicPartition.partition, s"$i".getBytes, s"$i".getBytes) + debug(s"Sending this record: $record") + producer.send(record) + }) + + futures.foreach(_.get) + } + + // TODO: ENABLE_FOR_HSTREAM: need to support alter config // @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) // @ValueSource(strings = Array("zk", "kraft")) // def testInvalidAlterConfigs(quorum: String): Unit = { @@ -1027,25 +1099,33 @@ // checkInvalidAlterConfigs(this, client) // } - /** - * Test that ACL operations are not possible when the authorizer is disabled. - * Also see [[kafka.api.SaslSslAdminIntegrationTest.testAclOperations()]] for tests of ACL operations - * when the authorizer is enabled. - */ - @Test - def testAclOperations(): Unit = { - val acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), - new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)) - client = Admin.create(createConfig) - assertFutureExceptionTypeEquals(client.describeAcls(AclBindingFilter.ANY).values(), classOf[SecurityDisabledException]) - assertFutureExceptionTypeEquals(client.createAcls(Collections.singleton(acl)).all(), - classOf[SecurityDisabledException]) - assertFutureExceptionTypeEquals(client.deleteAcls(Collections.singleton(acl.toFilter())).all(), - classOf[SecurityDisabledException]) - } - - - // KAFKA_TO_HSTREAM: Admin Client Test, not related to broker + /** + * Test that ACL operations are not possible when the authorizer is disabled. Also see + * [[kafka.api.SaslSslAdminIntegrationTest.testAclOperations()]] for tests of ACL operations when the authorizer is + * enabled. + */ + @Test + def testAclOperations(): Unit = { + val acl = new AclBinding( + new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), + new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW) + ) + client = Admin.create(createConfig) + assertFutureExceptionTypeEquals( + client.describeAcls(AclBindingFilter.ANY).values(), + classOf[SecurityDisabledException] + ) + assertFutureExceptionTypeEquals( + client.createAcls(Collections.singleton(acl)).all(), + classOf[SecurityDisabledException] + ) + assertFutureExceptionTypeEquals( + client.deleteAcls(Collections.singleton(acl.toFilter())).all(), + classOf[SecurityDisabledException] + ) + } + + // KAFKA_TO_HSTREAM: Admin Client Test, not related to broker // /** // * Test closing the AdminClient with a generous timeout. Calls in progress should be completed, // * since they can be done within the timeout. New calls should receive timeouts. @@ -1119,301 +1199,348 @@ // assertEquals(1, factory.failuresInjected) // } - /** - * Test the consumer group APIs. - */ - @Test - @Disabled("JoinGroupRequest should >= V5") - def testConsumerGroups(): Unit = { - val config = createConfig - client = Admin.create(config) - try { - // Verify that initially there are no consumer groups to list. - val list1 = client.listConsumerGroups() - assertTrue(0 == list1.all().get().size()) - assertTrue(0 == list1.errors().get().size()) - assertTrue(0 == list1.valid().get().size()) - val testTopicName = "test_topic" - val testTopicName1 = testTopicName + "1" - val testTopicName2 = testTopicName + "2" - val testNumPartitions = 2 - - client.createTopics(util.Arrays.asList( - new NewTopic(testTopicName, testNumPartitions, 1.toShort), - new NewTopic(testTopicName1, testNumPartitions, 1.toShort), - new NewTopic(testTopicName2, testNumPartitions, 1.toShort) - )).all().get() - waitForTopics(client, List(testTopicName, testTopicName1, testTopicName2), List()) - - val producer = createProducer() - try { - producer.send(new ProducerRecord(testTopicName, 0, null, null)).get() - } finally { - Utils.closeQuietly(producer, "producer") - } - - val EMPTY_GROUP_INSTANCE_ID = "" - val testGroupId = "test_group_id" - val testClientId = "test_client_id" - val testInstanceId1 = "test_instance_id_1" - val testInstanceId2 = "test_instance_id_2" - val fakeGroupId = "fake_group_id" - - def createProperties(groupInstanceId: String): Properties = { - val newConsumerConfig = new Properties(consumerConfig) - // We need to disable the auto commit because after the members got removed from group, the offset commit - // will cause the member rejoining and the test will be flaky (check ConsumerCoordinator#OffsetCommitResponseHandler) - newConsumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - newConsumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, testGroupId) - newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) - if (groupInstanceId != EMPTY_GROUP_INSTANCE_ID) { - newConsumerConfig.setProperty(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId) - } - newConsumerConfig - } - - // contains two static members and one dynamic member - val groupInstanceSet = Set(testInstanceId1, testInstanceId2, EMPTY_GROUP_INSTANCE_ID) - val consumerSet = groupInstanceSet.map { groupInstanceId => createConsumer(configOverrides = createProperties(groupInstanceId))} - val topicSet = Set(testTopicName, testTopicName1, testTopicName2) - - val latch = new CountDownLatch(consumerSet.size) - try { - def createConsumerThread[K,V](consumer: KafkaConsumer[K,V], topic: String): Thread = { - new Thread { - override def run : Unit = { - consumer.subscribe(Collections.singleton(topic)) - try { - while (true) { - consumer.poll(JDuration.ofSeconds(5)) - if (!consumer.assignment.isEmpty && latch.getCount > 0L) - latch.countDown() - consumer.commitSync() - } - } catch { - case _: InterruptException => // Suppress the output to stderr - } - } - } - } - - // Start consumers in a thread that will subscribe to a new group. - val consumerThreads = consumerSet.zip(topicSet).map(zipped => createConsumerThread(zipped._1, zipped._2)) - - try { - consumerThreads.foreach(_.start()) - assertTrue(latch.await(30000, TimeUnit.MILLISECONDS)) - // Test that we can list the new group. - TestUtils.waitUntilTrue(() => { - val matching = client.listConsumerGroups.all.get.asScala.filter(group => - group.groupId == testGroupId && - group.state.get == ConsumerGroupState.STABLE) - matching.size == 1 - }, s"Expected to be able to list $testGroupId") - - TestUtils.waitUntilTrue(() => { - val options = new ListConsumerGroupsOptions().inStates(Set(ConsumerGroupState.STABLE).asJava) - val matching = client.listConsumerGroups(options).all.get.asScala.filter(group => - group.groupId == testGroupId && - group.state.get == ConsumerGroupState.STABLE) - matching.size == 1 - }, s"Expected to be able to list $testGroupId in state Stable") - - TestUtils.waitUntilTrue(() => { - val options = new ListConsumerGroupsOptions().inStates(Set(ConsumerGroupState.EMPTY).asJava) - val matching = client.listConsumerGroups(options).all.get.asScala.filter( - _.groupId == testGroupId) - matching.isEmpty - }, s"Expected to find zero groups") - - val describeWithFakeGroupResult = client.describeConsumerGroups(Seq(testGroupId, fakeGroupId).asJava, - new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true)) - assertEquals(2, describeWithFakeGroupResult.describedGroups().size()) - - // Test that we can get information about the test consumer group. - assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(testGroupId)) - var testGroupDescription = describeWithFakeGroupResult.describedGroups().get(testGroupId).get() - - assertEquals(testGroupId, testGroupDescription.groupId()) - assertFalse(testGroupDescription.isSimpleConsumerGroup) - assertEquals(groupInstanceSet.size, testGroupDescription.members().size()) - val members = testGroupDescription.members() - members.asScala.foreach(member => assertEquals(testClientId, member.clientId())) - val topicPartitionsByTopic = members.asScala.flatMap(_.assignment().topicPartitions().asScala).groupBy(_.topic()) - topicSet.foreach { topic => - val topicPartitions = topicPartitionsByTopic.getOrElse(topic, List.empty) - assertEquals(testNumPartitions, topicPartitions.size) - } + /** + * Test the consumer group APIs. + */ + @Test + @Disabled("JoinGroupRequest should >= V5") + def testConsumerGroups(): Unit = { + val config = createConfig + client = Admin.create(config) + try { + // Verify that initially there are no consumer groups to list. + val list1 = client.listConsumerGroups() + assertTrue(0 == list1.all().get().size()) + assertTrue(0 == list1.errors().get().size()) + assertTrue(0 == list1.valid().get().size()) + val testTopicName = "test_topic" + val testTopicName1 = testTopicName + "1" + val testTopicName2 = testTopicName + "2" + val testNumPartitions = 2 + + client + .createTopics( + util.Arrays.asList( + new NewTopic(testTopicName, testNumPartitions, 1.toShort), + new NewTopic(testTopicName1, testNumPartitions, 1.toShort), + new NewTopic(testTopicName2, testNumPartitions, 1.toShort) + ) + ) + .all() + .get() + waitForTopics(client, List(testTopicName, testTopicName1, testTopicName2), List()) + + val producer = createProducer() + try { + producer.send(new ProducerRecord(testTopicName, 0, null, null)).get() + } finally { + Utils.closeQuietly(producer, "producer") + } + + val EMPTY_GROUP_INSTANCE_ID = "" + val testGroupId = "test_group_id" + val testClientId = "test_client_id" + val testInstanceId1 = "test_instance_id_1" + val testInstanceId2 = "test_instance_id_2" + val fakeGroupId = "fake_group_id" + + def createProperties(groupInstanceId: String): Properties = { + val newConsumerConfig = new Properties(consumerConfig) + // We need to disable the auto commit because after the members got removed from group, the offset commit + // will cause the member rejoining and the test will be flaky (check ConsumerCoordinator#OffsetCommitResponseHandler) + newConsumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + newConsumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, testGroupId) + newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) + if (groupInstanceId != EMPTY_GROUP_INSTANCE_ID) { + newConsumerConfig.setProperty(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId) + } + newConsumerConfig + } + + // contains two static members and one dynamic member + val groupInstanceSet = Set(testInstanceId1, testInstanceId2, EMPTY_GROUP_INSTANCE_ID) + val consumerSet = groupInstanceSet.map { groupInstanceId => + createConsumer(configOverrides = createProperties(groupInstanceId)) + } + val topicSet = Set(testTopicName, testTopicName1, testTopicName2) + + val latch = new CountDownLatch(consumerSet.size) + try { + def createConsumerThread[K, V](consumer: KafkaConsumer[K, V], topic: String): Thread = { + new Thread { + override def run: Unit = { + consumer.subscribe(Collections.singleton(topic)) + try { + while (true) { + consumer.poll(JDuration.ofSeconds(5)) + if (!consumer.assignment.isEmpty && latch.getCount > 0L) + latch.countDown() + consumer.commitSync() + } + } catch { + case _: InterruptException => // Suppress the output to stderr + } + } + } + } + + // Start consumers in a thread that will subscribe to a new group. + val consumerThreads = consumerSet.zip(topicSet).map(zipped => createConsumerThread(zipped._1, zipped._2)) + + try { + consumerThreads.foreach(_.start()) + assertTrue(latch.await(30000, TimeUnit.MILLISECONDS)) + // Test that we can list the new group. + TestUtils.waitUntilTrue( + () => { + val matching = client.listConsumerGroups.all.get.asScala.filter(group => + group.groupId == testGroupId && + group.state.get == ConsumerGroupState.STABLE + ) + matching.size == 1 + }, + s"Expected to be able to list $testGroupId" + ) + + TestUtils.waitUntilTrue( + () => { + val options = new ListConsumerGroupsOptions().inStates(Set(ConsumerGroupState.STABLE).asJava) + val matching = client + .listConsumerGroups(options) + .all + .get + .asScala + .filter(group => + group.groupId == testGroupId && + group.state.get == ConsumerGroupState.STABLE + ) + matching.size == 1 + }, + s"Expected to be able to list $testGroupId in state Stable" + ) + + TestUtils.waitUntilTrue( + () => { + val options = new ListConsumerGroupsOptions().inStates(Set(ConsumerGroupState.EMPTY).asJava) + val matching = client.listConsumerGroups(options).all.get.asScala.filter(_.groupId == testGroupId) + matching.isEmpty + }, + s"Expected to find zero groups" + ) + + val describeWithFakeGroupResult = client.describeConsumerGroups( + Seq(testGroupId, fakeGroupId).asJava, + new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true) + ) + assertEquals(2, describeWithFakeGroupResult.describedGroups().size()) + + // Test that we can get information about the test consumer group. + assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(testGroupId)) + var testGroupDescription = describeWithFakeGroupResult.describedGroups().get(testGroupId).get() + + assertEquals(testGroupId, testGroupDescription.groupId()) + assertFalse(testGroupDescription.isSimpleConsumerGroup) + assertEquals(groupInstanceSet.size, testGroupDescription.members().size()) + val members = testGroupDescription.members() + members.asScala.foreach(member => assertEquals(testClientId, member.clientId())) + val topicPartitionsByTopic = + members.asScala.flatMap(_.assignment().topicPartitions().asScala).groupBy(_.topic()) + topicSet.foreach { topic => + val topicPartitions = topicPartitionsByTopic.getOrElse(topic, List.empty) + assertEquals(testNumPartitions, topicPartitions.size) + } // val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP).asJava // assertEquals(expectedOperations, testGroupDescription.authorizedOperations()) - // Test that the fake group is listed as dead. - assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId)) - val fakeGroupDescription = describeWithFakeGroupResult.describedGroups().get(fakeGroupId).get() + // Test that the fake group is listed as dead. + assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId)) + val fakeGroupDescription = describeWithFakeGroupResult.describedGroups().get(fakeGroupId).get() - assertEquals(fakeGroupId, fakeGroupDescription.groupId()) - assertEquals(0, fakeGroupDescription.members().size()) - assertEquals("", fakeGroupDescription.partitionAssignor()) - assertEquals(ConsumerGroupState.DEAD, fakeGroupDescription.state()) + assertEquals(fakeGroupId, fakeGroupDescription.groupId()) + assertEquals(0, fakeGroupDescription.members().size()) + assertEquals("", fakeGroupDescription.partitionAssignor()) + assertEquals(ConsumerGroupState.DEAD, fakeGroupDescription.state()) // assertEquals(expectedOperations, fakeGroupDescription.authorizedOperations()) - // Test that all() returns 2 results - assertEquals(2, describeWithFakeGroupResult.all().get().size()) - - // Test listConsumerGroupOffsets - TestUtils.waitUntilTrue(() => { - val parts = client.listConsumerGroupOffsets(testGroupId).partitionsToOffsetAndMetadata().get() - val part = new TopicPartition(testTopicName, 0) - parts.containsKey(part) && (parts.get(part).offset() == 1) - }, s"Expected the offset for partition 0 to eventually become 1.") - - // Test delete non-exist consumer instance - val invalidInstanceId = "invalid-instance-id" - var removeMembersResult = client.removeMembersFromConsumerGroup(testGroupId, new RemoveMembersFromConsumerGroupOptions( - Collections.singleton(new MemberToRemove(invalidInstanceId)) - )) - - TestUtils.assertFutureExceptionTypeEquals(removeMembersResult.all, classOf[UnknownMemberIdException]) - val firstMemberFuture = removeMembersResult.memberResult(new MemberToRemove(invalidInstanceId)) - TestUtils.assertFutureExceptionTypeEquals(firstMemberFuture, classOf[UnknownMemberIdException]) - - // Test consumer group deletion - var deleteResult = client.deleteConsumerGroups(Seq(testGroupId, fakeGroupId).asJava) - assertEquals(2, deleteResult.deletedGroups().size()) - - // Deleting the fake group ID should get GroupIdNotFoundException. - assertTrue(deleteResult.deletedGroups().containsKey(fakeGroupId)) - assertFutureExceptionTypeEquals(deleteResult.deletedGroups().get(fakeGroupId), - classOf[GroupIdNotFoundException]) - - // Deleting the real group ID should get GroupNotEmptyException - assertTrue(deleteResult.deletedGroups().containsKey(testGroupId)) - assertFutureExceptionTypeEquals(deleteResult.deletedGroups().get(testGroupId), - classOf[GroupNotEmptyException]) - - // Test delete one correct static member - removeMembersResult = client.removeMembersFromConsumerGroup(testGroupId, new RemoveMembersFromConsumerGroupOptions( - Collections.singleton(new MemberToRemove(testInstanceId1)) - )) - - assertNull(removeMembersResult.all().get()) - val validMemberFuture = removeMembersResult.memberResult(new MemberToRemove(testInstanceId1)) - assertNull(validMemberFuture.get()) - - val describeTestGroupResult = client.describeConsumerGroups(Seq(testGroupId).asJava, - new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true)) - assertEquals(1, describeTestGroupResult.describedGroups().size()) - - testGroupDescription = describeTestGroupResult.describedGroups().get(testGroupId).get() - - assertEquals(testGroupId, testGroupDescription.groupId) - assertFalse(testGroupDescription.isSimpleConsumerGroup) - assertEquals(consumerSet.size - 1, testGroupDescription.members().size()) - - // Delete all active members remaining (a static member + a dynamic member) - removeMembersResult = client.removeMembersFromConsumerGroup(testGroupId, new RemoveMembersFromConsumerGroupOptions()) - assertNull(removeMembersResult.all().get()) - - // The group should contain no members now. - testGroupDescription = client.describeConsumerGroups(Seq(testGroupId).asJava, - new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true)) - .describedGroups().get(testGroupId).get() - assertTrue(testGroupDescription.members().isEmpty) - - // Consumer group deletion on empty group should succeed - deleteResult = client.deleteConsumerGroups(Seq(testGroupId).asJava) - assertEquals(1, deleteResult.deletedGroups().size()) - - assertTrue(deleteResult.deletedGroups().containsKey(testGroupId)) - assertNull(deleteResult.deletedGroups().get(testGroupId).get()) - } finally { - consumerThreads.foreach { - case consumerThread => - consumerThread.interrupt() - consumerThread.join() - } - } - } finally { - consumerSet.zip(groupInstanceSet).foreach(zipped => Utils.closeQuietly(zipped._1, zipped._2)) - } - } finally { - Utils.closeQuietly(client, "adminClient") - } - } - - @Test - @Disabled("require support OffsetDelete API") - def testDeleteConsumerGroupOffsets(): Unit = { - val config = createConfig - client = Admin.create(config) - try { - val testTopicName = "test_topic" - val testGroupId = "test_group_id" - val testClientId = "test_client_id" - val fakeGroupId = "fake_group_id" - - val tp1 = new TopicPartition(testTopicName, 0) - val tp2 = new TopicPartition("foo", 0) - - client.createTopics(Collections.singleton( - new NewTopic(testTopicName, 1, 1.toShort))).all().get() - waitForTopics(client, List(testTopicName), List()) - - val producer = createProducer() - try { - producer.send(new ProducerRecord(testTopicName, 0, null, null)).get() - } finally { - Utils.closeQuietly(producer, "producer") - } - - val newConsumerConfig = new Properties(consumerConfig) - newConsumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, testGroupId) - newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) - // Increase timeouts to avoid having a rebalance during the test - newConsumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, Integer.MAX_VALUE.toString) + // Test that all() returns 2 results + assertEquals(2, describeWithFakeGroupResult.all().get().size()) + + // Test listConsumerGroupOffsets + TestUtils.waitUntilTrue( + () => { + val parts = client.listConsumerGroupOffsets(testGroupId).partitionsToOffsetAndMetadata().get() + val part = new TopicPartition(testTopicName, 0) + parts.containsKey(part) && (parts.get(part).offset() == 1) + }, + s"Expected the offset for partition 0 to eventually become 1." + ) + + // Test delete non-exist consumer instance + val invalidInstanceId = "invalid-instance-id" + var removeMembersResult = client.removeMembersFromConsumerGroup( + testGroupId, + new RemoveMembersFromConsumerGroupOptions( + Collections.singleton(new MemberToRemove(invalidInstanceId)) + ) + ) + + TestUtils.assertFutureExceptionTypeEquals(removeMembersResult.all, classOf[UnknownMemberIdException]) + val firstMemberFuture = removeMembersResult.memberResult(new MemberToRemove(invalidInstanceId)) + TestUtils.assertFutureExceptionTypeEquals(firstMemberFuture, classOf[UnknownMemberIdException]) + + // Test consumer group deletion + var deleteResult = client.deleteConsumerGroups(Seq(testGroupId, fakeGroupId).asJava) + assertEquals(2, deleteResult.deletedGroups().size()) + + // Deleting the fake group ID should get GroupIdNotFoundException. + assertTrue(deleteResult.deletedGroups().containsKey(fakeGroupId)) + assertFutureExceptionTypeEquals( + deleteResult.deletedGroups().get(fakeGroupId), + classOf[GroupIdNotFoundException] + ) + + // Deleting the real group ID should get GroupNotEmptyException + assertTrue(deleteResult.deletedGroups().containsKey(testGroupId)) + assertFutureExceptionTypeEquals( + deleteResult.deletedGroups().get(testGroupId), + classOf[GroupNotEmptyException] + ) + + // Test delete one correct static member + removeMembersResult = client.removeMembersFromConsumerGroup( + testGroupId, + new RemoveMembersFromConsumerGroupOptions( + Collections.singleton(new MemberToRemove(testInstanceId1)) + ) + ) + + assertNull(removeMembersResult.all().get()) + val validMemberFuture = removeMembersResult.memberResult(new MemberToRemove(testInstanceId1)) + assertNull(validMemberFuture.get()) + + val describeTestGroupResult = client.describeConsumerGroups( + Seq(testGroupId).asJava, + new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true) + ) + assertEquals(1, describeTestGroupResult.describedGroups().size()) + + testGroupDescription = describeTestGroupResult.describedGroups().get(testGroupId).get() + + assertEquals(testGroupId, testGroupDescription.groupId) + assertFalse(testGroupDescription.isSimpleConsumerGroup) + assertEquals(consumerSet.size - 1, testGroupDescription.members().size()) + + // Delete all active members remaining (a static member + a dynamic member) + removeMembersResult = + client.removeMembersFromConsumerGroup(testGroupId, new RemoveMembersFromConsumerGroupOptions()) + assertNull(removeMembersResult.all().get()) + + // The group should contain no members now. + testGroupDescription = client + .describeConsumerGroups( + Seq(testGroupId).asJava, + new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true) + ) + .describedGroups() + .get(testGroupId) + .get() + assertTrue(testGroupDescription.members().isEmpty) + + // Consumer group deletion on empty group should succeed + deleteResult = client.deleteConsumerGroups(Seq(testGroupId).asJava) + assertEquals(1, deleteResult.deletedGroups().size()) + + assertTrue(deleteResult.deletedGroups().containsKey(testGroupId)) + assertNull(deleteResult.deletedGroups().get(testGroupId).get()) + } finally { + consumerThreads.foreach { case consumerThread => + consumerThread.interrupt() + consumerThread.join() + } + } + } finally { + consumerSet.zip(groupInstanceSet).foreach(zipped => Utils.closeQuietly(zipped._1, zipped._2)) + } + } finally { + Utils.closeQuietly(client, "adminClient") + } + } + + @Test + @Disabled("require support OffsetDelete API") + def testDeleteConsumerGroupOffsets(): Unit = { + val config = createConfig + client = Admin.create(config) + try { + val testTopicName = "test_topic" + val testGroupId = "test_group_id" + val testClientId = "test_client_id" + val fakeGroupId = "fake_group_id" + + val tp1 = new TopicPartition(testTopicName, 0) + val tp2 = new TopicPartition("foo", 0) + + client.createTopics(Collections.singleton(new NewTopic(testTopicName, 1, 1.toShort))).all().get() + waitForTopics(client, List(testTopicName), List()) + + val producer = createProducer() + try { + producer.send(new ProducerRecord(testTopicName, 0, null, null)).get() + } finally { + Utils.closeQuietly(producer, "producer") + } + + val newConsumerConfig = new Properties(consumerConfig) + newConsumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, testGroupId) + newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) + // Increase timeouts to avoid having a rebalance during the test + newConsumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, Integer.MAX_VALUE.toString) // newConsumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Defaults.GroupMaxSessionTimeoutMs.toString) - val consumer = createConsumer(configOverrides = newConsumerConfig) - - try { - TestUtils.subscribeAndWaitForRecords(testTopicName, consumer) - consumer.commitSync() - - // Test offset deletion while consuming - val offsetDeleteResult = client.deleteConsumerGroupOffsets(testGroupId, Set(tp1, tp2).asJava) - - // Top level error will equal to the first partition level error - assertFutureExceptionTypeEquals(offsetDeleteResult.all(), classOf[GroupSubscribedToTopicException]) - assertFutureExceptionTypeEquals(offsetDeleteResult.partitionResult(tp1), - classOf[GroupSubscribedToTopicException]) - assertFutureExceptionTypeEquals(offsetDeleteResult.partitionResult(tp2), - classOf[UnknownTopicOrPartitionException]) - - // Test the fake group ID - val fakeDeleteResult = client.deleteConsumerGroupOffsets(fakeGroupId, Set(tp1, tp2).asJava) - - assertFutureExceptionTypeEquals(fakeDeleteResult.all(), classOf[GroupIdNotFoundException]) - assertFutureExceptionTypeEquals(fakeDeleteResult.partitionResult(tp1), - classOf[GroupIdNotFoundException]) - assertFutureExceptionTypeEquals(fakeDeleteResult.partitionResult(tp2), - classOf[GroupIdNotFoundException]) - - } finally { - Utils.closeQuietly(consumer, "consumer") - } - - // Test offset deletion when group is empty - val offsetDeleteResult = client.deleteConsumerGroupOffsets(testGroupId, Set(tp1, tp2).asJava) - - assertFutureExceptionTypeEquals(offsetDeleteResult.all(), - classOf[UnknownTopicOrPartitionException]) - assertNull(offsetDeleteResult.partitionResult(tp1).get()) - assertFutureExceptionTypeEquals(offsetDeleteResult.partitionResult(tp2), - classOf[UnknownTopicOrPartitionException]) - } finally { - Utils.closeQuietly(client, "adminClient") - } - } + val consumer = createConsumer(configOverrides = newConsumerConfig) + + try { + TestUtils.subscribeAndWaitForRecords(testTopicName, consumer) + consumer.commitSync() + + // Test offset deletion while consuming + val offsetDeleteResult = client.deleteConsumerGroupOffsets(testGroupId, Set(tp1, tp2).asJava) + + // Top level error will equal to the first partition level error + assertFutureExceptionTypeEquals(offsetDeleteResult.all(), classOf[GroupSubscribedToTopicException]) + assertFutureExceptionTypeEquals( + offsetDeleteResult.partitionResult(tp1), + classOf[GroupSubscribedToTopicException] + ) + assertFutureExceptionTypeEquals( + offsetDeleteResult.partitionResult(tp2), + classOf[UnknownTopicOrPartitionException] + ) + + // Test the fake group ID + val fakeDeleteResult = client.deleteConsumerGroupOffsets(fakeGroupId, Set(tp1, tp2).asJava) + + assertFutureExceptionTypeEquals(fakeDeleteResult.all(), classOf[GroupIdNotFoundException]) + assertFutureExceptionTypeEquals(fakeDeleteResult.partitionResult(tp1), classOf[GroupIdNotFoundException]) + assertFutureExceptionTypeEquals(fakeDeleteResult.partitionResult(tp2), classOf[GroupIdNotFoundException]) + + } finally { + Utils.closeQuietly(consumer, "consumer") + } + + // Test offset deletion when group is empty + val offsetDeleteResult = client.deleteConsumerGroupOffsets(testGroupId, Set(tp1, tp2).asJava) + + assertFutureExceptionTypeEquals(offsetDeleteResult.all(), classOf[UnknownTopicOrPartitionException]) + assertNull(offsetDeleteResult.partitionResult(tp1).get()) + assertFutureExceptionTypeEquals( + offsetDeleteResult.partitionResult(tp2), + classOf[UnknownTopicOrPartitionException] + ) + } finally { + Utils.closeQuietly(client, "adminClient") + } + } // @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) // @ValueSource(strings = Array("zk", "kraft")) @@ -1794,22 +1921,22 @@ // TestUtils.assertLeader(client, partition2, broker3) // } // - @Test - @Disabled("require support ListPartitionReassignments API") - def testListReassignmentsDoesNotShowNonReassigningPartitions(): Unit = { - client = Admin.create(createConfig) + @Test + @Disabled("require support ListPartitionReassignments API") + def testListReassignmentsDoesNotShowNonReassigningPartitions(): Unit = { + client = Admin.create(createConfig) - // Create topics - val topic = "list-reassignments-no-reassignments" - createTopic(topic, replicationFactor = 3) - val tp = new TopicPartition(topic, 0) + // Create topics + val topic = "list-reassignments-no-reassignments" + createTopic(topic, replicationFactor = 3) + val tp = new TopicPartition(topic, 0) - val reassignmentsMap = client.listPartitionReassignments(Set(tp).asJava).reassignments().get() - assertEquals(0, reassignmentsMap.size()) + val reassignmentsMap = client.listPartitionReassignments(Set(tp).asJava).reassignments().get() + assertEquals(0, reassignmentsMap.size()) - val allReassignmentsMap = client.listPartitionReassignments().reassignments().get() - assertEquals(0, allReassignmentsMap.size()) - } + val allReassignmentsMap = client.listPartitionReassignments().reassignments().get() + assertEquals(0, allReassignmentsMap.size()) + } // // @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) // @ValueSource(strings = Array("zk", "kraft")) @@ -2194,24 +2321,23 @@ // assertFutureExceptionTypeEquals(invalidReplicaResult.get(tp3), classOf[InvalidReplicaAssignmentException]) // } // - @Test - def testLongTopicNames(): Unit = { - val client = Admin.create(createConfig) - val longTopicName = String.join("", Collections.nCopies(249, "x")); - val invalidTopicName = String.join("", Collections.nCopies(250, "x")); - val newTopics2 = Seq(new NewTopic(invalidTopicName, 3, 3.toShort), - new NewTopic(longTopicName, 3, 3.toShort)) - val results = client.createTopics(newTopics2.asJava).values() - assertTrue(results.containsKey(longTopicName)) - results.get(longTopicName).get() - assertTrue(results.containsKey(invalidTopicName)) - assertFutureExceptionTypeEquals(results.get(invalidTopicName), classOf[InvalidTopicException]) - // KAFKA_TO_HSTREAM + @Test + def testLongTopicNames(): Unit = { + val client = Admin.create(createConfig) + val longTopicName = String.join("", Collections.nCopies(249, "x")); + val invalidTopicName = String.join("", Collections.nCopies(250, "x")); + val newTopics2 = Seq(new NewTopic(invalidTopicName, 3, 3.toShort), new NewTopic(longTopicName, 3, 3.toShort)) + val results = client.createTopics(newTopics2.asJava).values() + assertTrue(results.containsKey(longTopicName)) + results.get(longTopicName).get() + assertTrue(results.containsKey(invalidTopicName)) + assertFutureExceptionTypeEquals(results.get(invalidTopicName), classOf[InvalidTopicException]) + // KAFKA_TO_HSTREAM // assertFutureExceptionTypeEquals(client.alterReplicaLogDirs( // Map(new TopicPartitionReplica(longTopicName, 0, 0) -> brokers(0).config.logDirs(0)).asJava).all(), // classOf[InvalidTopicException]) - client.close() - } + client.close() + } // // // Verify that createTopics and alterConfigs fail with null values // @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @@ -2773,4 +2899,4 @@ // // assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) // } - } +} diff --git a/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index f45424e..dad40df 100644 --- a/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/app/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -1,14 +1,18 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.api diff --git a/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index fe9feca..da9740c 100644 --- a/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/app/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -1,22 +1,30 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.api import kafka.utils.TestUtils import org.apache.kafka.clients.producer._ import org.apache.kafka.common.config.TopicConfig -import org.apache.kafka.common.errors.{InvalidTimestampException, RecordTooLargeException, SerializationException, TimeoutException} +import org.apache.kafka.common.errors.{ + InvalidTimestampException, + RecordTooLargeException, + SerializationException, + TimeoutException +} import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch, Records, TimestampType} import org.apache.kafka.common.serialization.ByteArraySerializer diff --git a/app/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/app/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index d0a2c77..edf4a21 100755 --- a/app/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/app/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -1,16 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.api.test import kafka.server.{KafkaBroker, KafkaConfig, QuorumTestHarness} diff --git a/app/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/app/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 519b136..6726574 100644 --- a/app/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/app/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -1,16 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.api import java.util.concurrent.ExecutionException diff --git a/app/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/app/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index ae74034..a05e4ba 100644 --- a/app/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/app/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -1,14 +1,18 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.api diff --git a/app/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/app/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index 71d81b0..eb3f816 100644 --- a/app/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/app/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -1,16 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.server import java.util diff --git a/app/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/app/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index 44660fb..515d447 100644 --- a/app/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/app/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -1,16 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.server import kafka.utils._ diff --git a/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index 48bb27f..4c00836 100644 --- a/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/app/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -1,254 +1,285 @@ - /** - * 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.server - - import kafka.utils.TestUtils - import org.apache.kafka.clients.consumer.{ConsumerConfig, OffsetAndMetadata} - import org.apache.kafka.common.TopicPartition - import org.apache.kafka.common.protocol.{ApiKeys, Errors} - import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData - import org.apache.kafka.common.requests.{AbstractResponse, OffsetFetchRequest, OffsetFetchResponse} - import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} - import org.junit.jupiter.api.{BeforeEach, Disabled, Test, TestInfo} - - import java.util - import java.util.Collections.singletonList - import scala.jdk.CollectionConverters._ - import java.util.{Collections, Optional, Properties} - - class OffsetFetchRequestTest extends BaseRequestTest { - - override def brokerCount: Int = 1 - - val brokerId: Integer = 0 - val offset = 15L - val leaderEpoch: Optional[Integer] = Optional.of(3) - val metadata = "metadata" - val topic = "topic" - val groupId = "groupId" - val groups: Seq[String] = (1 to 5).map(i => s"group$i") - val topics: Seq[String] = (1 to 3).map(i => s"topic$i") - val topic1List = singletonList(new TopicPartition(topics(0), 0)) - val topic1And2List = util.Arrays.asList( - new TopicPartition(topics(0), 0), - new TopicPartition(topics(1), 0), - new TopicPartition(topics(1), 1)) - val allTopicsList = util.Arrays.asList( - new TopicPartition(topics(0), 0), - new TopicPartition(topics(1), 0), - new TopicPartition(topics(1), 1), - new TopicPartition(topics(2), 0), - new TopicPartition(topics(2), 1), - new TopicPartition(topics(2), 2)) - val groupToPartitionMap: util.Map[String, util.List[TopicPartition]] = - new util.HashMap[String, util.List[TopicPartition]]() - groupToPartitionMap.put(groups(0), topic1List) - groupToPartitionMap.put(groups(1), topic1And2List) - groupToPartitionMap.put(groups(2), allTopicsList) - groupToPartitionMap.put(groups(3), null) - groupToPartitionMap.put(groups(4), null) - - override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) - // TODO: ENABLE_FOR_HSTREAM +/* + * 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.server + +import kafka.utils.TestUtils +import org.apache.kafka.clients.consumer.{ConsumerConfig, OffsetAndMetadata} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData +import org.apache.kafka.common.requests.{AbstractResponse, OffsetFetchRequest, OffsetFetchResponse} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.{BeforeEach, Disabled, Test, TestInfo} + +import java.util +import java.util.Collections.singletonList +import scala.jdk.CollectionConverters._ +import java.util.{Collections, Optional, Properties} + +class OffsetFetchRequestTest extends BaseRequestTest { + + override def brokerCount: Int = 1 + + val brokerId: Integer = 0 + val offset = 15L + val leaderEpoch: Optional[Integer] = Optional.of(3) + val metadata = "metadata" + val topic = "topic" + val groupId = "groupId" + val groups: Seq[String] = (1 to 5).map(i => s"group$i") + val topics: Seq[String] = (1 to 3).map(i => s"topic$i") + val topic1List = singletonList(new TopicPartition(topics(0), 0)) + val topic1And2List = util.Arrays.asList( + new TopicPartition(topics(0), 0), + new TopicPartition(topics(1), 0), + new TopicPartition(topics(1), 1) + ) + val allTopicsList = util.Arrays.asList( + new TopicPartition(topics(0), 0), + new TopicPartition(topics(1), 0), + new TopicPartition(topics(1), 1), + new TopicPartition(topics(2), 0), + new TopicPartition(topics(2), 1), + new TopicPartition(topics(2), 2) + ) + val groupToPartitionMap: util.Map[String, util.List[TopicPartition]] = + new util.HashMap[String, util.List[TopicPartition]]() + groupToPartitionMap.put(groups(0), topic1List) + groupToPartitionMap.put(groups(1), topic1And2List) + groupToPartitionMap.put(groups(2), allTopicsList) + groupToPartitionMap.put(groups(3), null) + groupToPartitionMap.put(groups(4), null) + + override def brokerPropertyOverrides(properties: Properties): Unit = { + properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + // TODO: ENABLE_FOR_HSTREAM // properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") - properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") + properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") // properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1") // properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1") // properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1") - } - - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - doSetup(testInfo, createOffsetsTopic = false) - - // ENABLE_FOR_HSTREAM: currently HStream doesn't support creating offsets topic - // TestUtils.createOffsetsTopic(zkClient, servers) - } - - @Test - def testOffsetFetchRequestSingleGroup(): Unit = { - createTopic(topic) - - val tpList = singletonList(new TopicPartition(topic, 0)) - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) - commitOffsets(tpList) - - // testing from version 1 onward since version 0 read offsets from ZK - // TODO: ENABLE_FOR_HSTREAM - // for (version <- 1 to ApiKeys.OFFSET_FETCH.latestVersion()) { - for (version <- 1 to 3) { - if (version < 8) { - val request = - if (version < 7) { - new OffsetFetchRequest.Builder( - groupId, false, tpList, false) - .build(version.asInstanceOf[Short]) - } else { - new OffsetFetchRequest.Builder( - groupId, false, tpList, true) - .build(version.asInstanceOf[Short]) - } - val response = connectAndReceive[OffsetFetchResponse](request) - val topicData = response.data().topics().get(0) - val partitionData = topicData.partitions().get(0) - if (version < 3) { - assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs()) - } - verifySingleGroupResponse(version.asInstanceOf[Short], - response.error().code(), partitionData.errorCode(), topicData.name(), - partitionData.partitionIndex(), partitionData.committedOffset(), - partitionData.committedLeaderEpoch(), partitionData.metadata()) - } else { - val request = new OffsetFetchRequest.Builder( - Map(groupId -> tpList).asJava, false, false) - .build(version.asInstanceOf[Short]) - val response = connectAndReceive[OffsetFetchResponse](request) - val groupData = response.data().groups().get(0) - val topicData = groupData.topics().get(0) - val partitionData = topicData.partitions().get(0) - verifySingleGroupResponse(version.asInstanceOf[Short], - groupData.errorCode(), partitionData.errorCode(), topicData.name(), - partitionData.partitionIndex(), partitionData.committedOffset(), - partitionData.committedLeaderEpoch(), partitionData.metadata()) - } - } - } - - @Test - def testOffsetFetchRequestAllOffsetsSingleGroup(): Unit = { - createTopic(topic) - - val tpList = singletonList(new TopicPartition(topic, 0)) - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) - commitOffsets(tpList) - - // Testing from version 2 onward since version 0 and do not support - // fetching all offsets. - // TODO: ENABLE_FOR_HSTREAM - // for (version <- 2 to ApiKeys.OFFSET_FETCH.latestVersion()) { - for (version <- 2 to 3) { - if (version < 8) { - val request = new OffsetFetchRequest.Builder( - groupId, - false, - null, - version >= 7 - ).build(version.toShort) - - val response = connectAndReceive[OffsetFetchResponse](request) - assertEquals(Errors.NONE, response.error()) - val topicData = response.data.topics().get(0) - val partitionData = topicData.partitions().get(0) - if (version < 3) { - assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs()) - } - verifySingleGroupResponse(version.asInstanceOf[Short], - response.error().code(), partitionData.errorCode(), topicData.name(), - partitionData.partitionIndex(), partitionData.committedOffset(), - partitionData.committedLeaderEpoch(), partitionData.metadata()) - } else { - val request = new OffsetFetchRequest.Builder( - Collections.singletonMap(groupId, null), - false, - false - ).build(version.toShort) - - val response = connectAndReceive[OffsetFetchResponse](request) - assertEquals(Errors.NONE, response.groupLevelError(groupId)) - val groupData = response.data().groups().get(0) - val topicData = groupData.topics().get(0) - val partitionData = topicData.partitions().get(0) - verifySingleGroupResponse(version.asInstanceOf[Short], - groupData.errorCode(), partitionData.errorCode(), topicData.name(), - partitionData.partitionIndex(), partitionData.committedOffset(), - partitionData.committedLeaderEpoch(), partitionData.metadata()) - } - } - } - - @Test - @Disabled("TODO: ENABLE_FOR_HSTREAM, need to support OffsetFetchRequest >= V8") - def testOffsetFetchRequestWithMultipleGroups(): Unit = { - createTopic(topics(0)) - createTopic(topics(1), numPartitions = 2) - createTopic(topics(2), numPartitions = 3) - - // create 5 consumers to commit offsets so we can fetch them later - val partitionMap = groupToPartitionMap.asScala.map(e => (e._1, Option(e._2).getOrElse(allTopicsList))) - groups.foreach { groupId => - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) - commitOffsets(partitionMap(groupId)) - } - - for (version <- 8 to ApiKeys.OFFSET_FETCH.latestVersion()) { - val request = new OffsetFetchRequest.Builder(groupToPartitionMap, false, false) - .build(version.asInstanceOf[Short]) - val response = connectAndReceive[OffsetFetchResponse](request) - response.data.groups.asScala.map(_.groupId).foreach( groupId => - verifyResponse(response.groupLevelError(groupId), response.partitionDataMap(groupId), partitionMap(groupId)) - ) - } - } - - private def verifySingleGroupResponse(version: Short, - responseError: Short, - partitionError: Short, - topicName: String, - partitionIndex: Integer, - committedOffset: Long, - committedLeaderEpoch: Integer, - partitionMetadata: String): Unit = { - assertEquals(Errors.NONE.code(), responseError) - assertEquals(topic, topicName) - assertEquals(0, partitionIndex) - assertEquals(offset, committedOffset) - if (version >= 5) { - assertEquals(leaderEpoch.get(), committedLeaderEpoch) - } - - // TODO: ENABLE_FOR_HSTREAM, currently metadata is not stored when commit - // assertEquals(metadata, partitionMetadata) - assertEquals(Errors.NONE.code(), partitionError) - } - - private def verifyPartitionData(partitionData: OffsetFetchResponse.PartitionData): Unit = { - assertTrue(!partitionData.hasError) - assertEquals(offset, partitionData.offset) - assertEquals(metadata, partitionData.metadata) - assertEquals(leaderEpoch.get(), partitionData.leaderEpoch.get()) - } - - private def verifyResponse(groupLevelResponse: Errors, - partitionData: util.Map[TopicPartition, PartitionData], - topicList: util.List[TopicPartition]): Unit = { - assertEquals(Errors.NONE, groupLevelResponse) - assertTrue(partitionData.size() == topicList.size()) - topicList.forEach(t => verifyPartitionData(partitionData.get(t))) - } - - private def commitOffsets(tpList: util.List[TopicPartition]): Unit = { - val consumer = createConsumer() - consumer.assign(tpList) - val offsets = tpList.asScala.map{ - tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata)) - }.toMap.asJava - consumer.commitSync(offsets) - consumer.close() - } - } + } + + @BeforeEach + override def setUp(testInfo: TestInfo): Unit = { + doSetup(testInfo, createOffsetsTopic = false) + + // ENABLE_FOR_HSTREAM: currently HStream doesn't support creating offsets topic + // TestUtils.createOffsetsTopic(zkClient, servers) + } + + @Test + def testOffsetFetchRequestSingleGroup(): Unit = { + createTopic(topic) + + val tpList = singletonList(new TopicPartition(topic, 0)) + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) + commitOffsets(tpList) + + // testing from version 1 onward since version 0 read offsets from ZK + // TODO: ENABLE_FOR_HSTREAM + // for (version <- 1 to ApiKeys.OFFSET_FETCH.latestVersion()) { + for (version <- 1 to 3) { + if (version < 8) { + val request = + if (version < 7) { + new OffsetFetchRequest.Builder(groupId, false, tpList, false) + .build(version.asInstanceOf[Short]) + } else { + new OffsetFetchRequest.Builder(groupId, false, tpList, true) + .build(version.asInstanceOf[Short]) + } + val response = connectAndReceive[OffsetFetchResponse](request) + val topicData = response.data().topics().get(0) + val partitionData = topicData.partitions().get(0) + if (version < 3) { + assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs()) + } + verifySingleGroupResponse( + version.asInstanceOf[Short], + response.error().code(), + partitionData.errorCode(), + topicData.name(), + partitionData.partitionIndex(), + partitionData.committedOffset(), + partitionData.committedLeaderEpoch(), + partitionData.metadata() + ) + } else { + val request = new OffsetFetchRequest.Builder(Map(groupId -> tpList).asJava, false, false) + .build(version.asInstanceOf[Short]) + val response = connectAndReceive[OffsetFetchResponse](request) + val groupData = response.data().groups().get(0) + val topicData = groupData.topics().get(0) + val partitionData = topicData.partitions().get(0) + verifySingleGroupResponse( + version.asInstanceOf[Short], + groupData.errorCode(), + partitionData.errorCode(), + topicData.name(), + partitionData.partitionIndex(), + partitionData.committedOffset(), + partitionData.committedLeaderEpoch(), + partitionData.metadata() + ) + } + } + } + + @Test + def testOffsetFetchRequestAllOffsetsSingleGroup(): Unit = { + createTopic(topic) + + val tpList = singletonList(new TopicPartition(topic, 0)) + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) + commitOffsets(tpList) + + // Testing from version 2 onward since version 0 and do not support + // fetching all offsets. + // TODO: ENABLE_FOR_HSTREAM + // for (version <- 2 to ApiKeys.OFFSET_FETCH.latestVersion()) { + for (version <- 2 to 3) { + if (version < 8) { + val request = new OffsetFetchRequest.Builder( + groupId, + false, + null, + version >= 7 + ).build(version.toShort) + + val response = connectAndReceive[OffsetFetchResponse](request) + assertEquals(Errors.NONE, response.error()) + val topicData = response.data.topics().get(0) + val partitionData = topicData.partitions().get(0) + if (version < 3) { + assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs()) + } + verifySingleGroupResponse( + version.asInstanceOf[Short], + response.error().code(), + partitionData.errorCode(), + topicData.name(), + partitionData.partitionIndex(), + partitionData.committedOffset(), + partitionData.committedLeaderEpoch(), + partitionData.metadata() + ) + } else { + val request = new OffsetFetchRequest.Builder( + Collections.singletonMap(groupId, null), + false, + false + ).build(version.toShort) + + val response = connectAndReceive[OffsetFetchResponse](request) + assertEquals(Errors.NONE, response.groupLevelError(groupId)) + val groupData = response.data().groups().get(0) + val topicData = groupData.topics().get(0) + val partitionData = topicData.partitions().get(0) + verifySingleGroupResponse( + version.asInstanceOf[Short], + groupData.errorCode(), + partitionData.errorCode(), + topicData.name(), + partitionData.partitionIndex(), + partitionData.committedOffset(), + partitionData.committedLeaderEpoch(), + partitionData.metadata() + ) + } + } + } + + @Test + @Disabled("TODO: ENABLE_FOR_HSTREAM, need to support OffsetFetchRequest >= V8") + def testOffsetFetchRequestWithMultipleGroups(): Unit = { + createTopic(topics(0)) + createTopic(topics(1), numPartitions = 2) + createTopic(topics(2), numPartitions = 3) + + // create 5 consumers to commit offsets so we can fetch them later + val partitionMap = groupToPartitionMap.asScala.map(e => (e._1, Option(e._2).getOrElse(allTopicsList))) + groups.foreach { groupId => + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) + commitOffsets(partitionMap(groupId)) + } + + for (version <- 8 to ApiKeys.OFFSET_FETCH.latestVersion()) { + val request = new OffsetFetchRequest.Builder(groupToPartitionMap, false, false) + .build(version.asInstanceOf[Short]) + val response = connectAndReceive[OffsetFetchResponse](request) + response.data.groups.asScala + .map(_.groupId) + .foreach(groupId => + verifyResponse(response.groupLevelError(groupId), response.partitionDataMap(groupId), partitionMap(groupId)) + ) + } + } + + private def verifySingleGroupResponse( + version: Short, + responseError: Short, + partitionError: Short, + topicName: String, + partitionIndex: Integer, + committedOffset: Long, + committedLeaderEpoch: Integer, + partitionMetadata: String + ): Unit = { + assertEquals(Errors.NONE.code(), responseError) + assertEquals(topic, topicName) + assertEquals(0, partitionIndex) + assertEquals(offset, committedOffset) + if (version >= 5) { + assertEquals(leaderEpoch.get(), committedLeaderEpoch) + } + + // TODO: ENABLE_FOR_HSTREAM, currently metadata is not stored when commit + // assertEquals(metadata, partitionMetadata) + assertEquals(Errors.NONE.code(), partitionError) + } + + private def verifyPartitionData(partitionData: OffsetFetchResponse.PartitionData): Unit = { + assertTrue(!partitionData.hasError) + assertEquals(offset, partitionData.offset) + assertEquals(metadata, partitionData.metadata) + assertEquals(leaderEpoch.get(), partitionData.leaderEpoch.get()) + } + + private def verifyResponse( + groupLevelResponse: Errors, + partitionData: util.Map[TopicPartition, PartitionData], + topicList: util.List[TopicPartition] + ): Unit = { + assertEquals(Errors.NONE, groupLevelResponse) + assertTrue(partitionData.size() == topicList.size()) + topicList.forEach(t => verifyPartitionData(partitionData.get(t))) + } + + private def commitOffsets(tpList: util.List[TopicPartition]): Unit = { + val consumer = createConsumer() + consumer.assign(tpList) + val offsets = tpList.asScala + .map { tp => + (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata)) + } + .toMap + .asJava + consumer.commitSync(offsets) + consumer.close() + } +} diff --git a/app/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/app/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index 3bce3c8..c76ab2b 100644 --- a/app/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/app/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -1,16 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.server import java.nio.ByteBuffer diff --git a/app/src/test/scala/utils/kafka/BaseFetchRequestTest.scala b/app/src/test/scala/utils/kafka/BaseFetchRequestTest.scala index 2535098..23878cb 100644 --- a/app/src/test/scala/utils/kafka/BaseFetchRequestTest.scala +++ b/app/src/test/scala/utils/kafka/BaseFetchRequestTest.scala @@ -1,14 +1,18 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.server diff --git a/app/src/test/scala/utils/kafka/BaseRequestTest.scala b/app/src/test/scala/utils/kafka/BaseRequestTest.scala index 144da63..08dba75 100644 --- a/app/src/test/scala/utils/kafka/BaseRequestTest.scala +++ b/app/src/test/scala/utils/kafka/BaseRequestTest.scala @@ -1,16 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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.server import kafka.api.IntegrationTestHarness diff --git a/app/src/test/scala/utils/kafka/IntegrationTestHarness.scala b/app/src/test/scala/utils/kafka/IntegrationTestHarness.scala index 840efc9..add5485 100644 --- a/app/src/test/scala/utils/kafka/IntegrationTestHarness.scala +++ b/app/src/test/scala/utils/kafka/IntegrationTestHarness.scala @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,8 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -// From: scala/integration/kafka/api/IntegrationTestHarness.scala package kafka.api import java.time.Duration @@ -127,8 +125,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { superuserClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) } - def doSetup(testInfo: TestInfo, - createOffsetsTopic: Boolean): Unit = { + def doSetup(testInfo: TestInfo, createOffsetsTopic: Boolean): Unit = { this.testInfo = testInfo // Generate client security properties before starting the brokers in case certs are needed producerConfig ++= clientSecurityProps("producer") @@ -160,17 +157,25 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } def clientSecurityProps(certAlias: String): Properties = { - TestUtils.securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, certAlias, TestUtils.SslCertificateCn, - clientSaslProperties) + TestUtils.securityConfigs( + Mode.CLIENT, + securityProtocol, + trustStoreFile, + certAlias, + TestUtils.SslCertificateCn, + clientSaslProperties + ) } def superuserSecurityProps(certAlias: String): Properties = { clientSecurityProps(certAlias) } - def createProducer[K, V](keySerializer: Serializer[K] = new ByteArraySerializer, - valueSerializer: Serializer[V] = new ByteArraySerializer, - configOverrides: Properties = new Properties): KafkaProducer[K, V] = { + def createProducer[K, V]( + keySerializer: Serializer[K] = new ByteArraySerializer, + valueSerializer: Serializer[V] = new ByteArraySerializer, + configOverrides: Properties = new Properties + ): KafkaProducer[K, V] = { val props = new Properties props ++= producerConfig props ++= configOverrides @@ -179,10 +184,12 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { producer } - def createAsyncConsumer[K, V](keyDeserializer: Deserializer[K] = new ByteArrayDeserializer, - valueDeserializer: Deserializer[V] = new ByteArrayDeserializer, - configOverrides: Properties = new Properties, - configsToRemove: List[String] = List()): PrototypeAsyncConsumer[K, V] = { + def createAsyncConsumer[K, V]( + keyDeserializer: Deserializer[K] = new ByteArrayDeserializer, + valueDeserializer: Deserializer[V] = new ByteArrayDeserializer, + configOverrides: Properties = new Properties, + configsToRemove: List[String] = List() + ): PrototypeAsyncConsumer[K, V] = { val props = new Properties props ++= consumerConfig props ++= configOverrides @@ -192,10 +199,12 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { consumer } - def createConsumer[K, V](keyDeserializer: Deserializer[K] = new ByteArrayDeserializer, - valueDeserializer: Deserializer[V] = new ByteArrayDeserializer, - configOverrides: Properties = new Properties, - configsToRemove: List[String] = List()): KafkaConsumer[K, V] = { + def createConsumer[K, V]( + keyDeserializer: Deserializer[K] = new ByteArrayDeserializer, + valueDeserializer: Deserializer[V] = new ByteArrayDeserializer, + configOverrides: Properties = new Properties, + configsToRemove: List[String] = List() + ): KafkaConsumer[K, V] = { val props = new Properties props ++= consumerConfig props ++= configOverrides @@ -206,8 +215,8 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } def createAdminClient( - listenerName: ListenerName = listenerName, - configOverrides: Properties = new Properties + listenerName: ListenerName = listenerName, + configOverrides: Properties = new Properties ): Admin = { val props = new Properties props ++= adminClientConfig @@ -218,8 +227,8 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } def createSuperuserAdminClient( - listenerName: ListenerName = listenerName, - configOverrides: Properties = new Properties + listenerName: ListenerName = listenerName, + configOverrides: Properties = new Properties ): Admin = { val props = new Properties props ++= superuserClientConfig diff --git a/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala b/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala index d7478b2..2275fbd 100644 --- a/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala +++ b/app/src/test/scala/utils/kafka/KafkaServerTestHarness.scala @@ -1,17 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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. */ - -// From: scala/unit/kafka/integration/KafkaServerTestHarness.scala package kafka.integration import kafka.network.SocketServer @@ -32,7 +34,7 @@ import java.net.Socket import java.nio.ByteBuffer import java.util.{Arrays, Properties} import scala.annotation.nowarn -import scala.collection.{Seq, mutable} +import scala.collection.{mutable, Seq} import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag // import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT @@ -373,8 +375,8 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { // KAFKA_TO_HSTREAM: move from: BaseRequestTest.scala def receive[T <: AbstractResponse](socket: Socket, apiKey: ApiKeys, version: Short)(implicit - classTag: ClassTag[T], - @nowarn("cat=unused") nn: NotNothing[T] + classTag: ClassTag[T], + @nowarn("cat=unused") nn: NotNothing[T] ): T = { val incoming = new DataInputStream(socket.getInputStream) val len = incoming.readInt() @@ -396,21 +398,21 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { // KAFKA_TO_HSTREAM: move from: BaseRequestTest.scala def sendAndReceive[T <: AbstractResponse]( - request: AbstractRequest, - socket: Socket, - clientId: String = "client-id", - correlationId: Option[Int] = None - )(implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { + request: AbstractRequest, + socket: Socket, + clientId: String = "client-id", + correlationId: Option[Int] = None + )(implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { send(request, socket, clientId, correlationId) receive[T](socket, request.apiKey, request.version) } // KAFKA_TO_HSTREAM: move from: BaseRequestTest.scala def connectAndReceive[T <: AbstractResponse]( - request: AbstractRequest, - destination: SocketServer = anySocketServer, - listenerName: ListenerName = listenerName - )(implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { + request: AbstractRequest, + destination: SocketServer = anySocketServer, + listenerName: ListenerName = listenerName + )(implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { val socket = connect(destination, listenerName) try sendAndReceive[T](request, socket) finally socket.close() @@ -421,11 +423,11 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { * Serializes and sends the request to the given api. */ def send( - request: AbstractRequest, - socket: Socket, - clientId: String = "client-id", - correlationId: Option[Int] = None - ): Unit = { + request: AbstractRequest, + socket: Socket, + clientId: String = "client-id", + correlationId: Option[Int] = None + ): Unit = { val header = nextRequestHeader(request.apiKey, request.version, clientId, correlationId) sendWithHeader(request, header, socket) } @@ -438,11 +440,11 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { // KAFKA_TO_HSTREAM: move from: BaseRequestTest.scala def nextRequestHeader[T <: AbstractResponse]( - apiKey: ApiKeys, - apiVersion: Short, - clientId: String = "client-id", - correlationIdOpt: Option[Int] = None - ): RequestHeader = { + apiKey: ApiKeys, + apiVersion: Short, + clientId: String = "client-id", + correlationIdOpt: Option[Int] = None + ): RequestHeader = { val correlationId = correlationIdOpt.getOrElse { this.correlationId += 1 this.correlationId diff --git a/app/src/test/scala/utils/kafka/QuorumTestHarness.scala b/app/src/test/scala/utils/kafka/QuorumTestHarness.scala index 17a9c15..f95a827 100755 --- a/app/src/test/scala/utils/kafka/QuorumTestHarness.scala +++ b/app/src/test/scala/utils/kafka/QuorumTestHarness.scala @@ -1,17 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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. */ - -// From: scala/integration/kafka/server/QuorumTestHarness.scala package kafka.server import kafka.utils.TestInfoUtils @@ -171,9 +173,9 @@ abstract class QuorumTestHarness extends Logging { // val bootstrapRecords: ListBuffer[ApiMessageAndVersion] = ListBuffer() // - def isKRaftTest(): Boolean = { - TestInfoUtils.isKRaft(testInfo) - } + def isKRaftTest(): Boolean = { + TestInfoUtils.isKRaft(testInfo) + } // // def isZkMigrationTest(): Boolean = { // TestInfoUtils.isZkMigrationTest(testInfo) diff --git a/app/src/test/scala/utils/kafka/utils/JaasTestUtils.scala b/app/src/test/scala/utils/kafka/utils/JaasTestUtils.scala index e4d61cc..c7a1755 100644 --- a/app/src/test/scala/utils/kafka/utils/JaasTestUtils.scala +++ b/app/src/test/scala/utils/kafka/utils/JaasTestUtils.scala @@ -1,17 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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. */ - -// From: scala/unit/kafka/utils/JaasTestUtils.scala package kafka.utils import java.io.{BufferedWriter, File, FileWriter} diff --git a/app/src/test/scala/utils/kafka/utils/TestInfoUtils.scala b/app/src/test/scala/utils/kafka/utils/TestInfoUtils.scala index fd3de0f..d12509d 100644 --- a/app/src/test/scala/utils/kafka/utils/TestInfoUtils.scala +++ b/app/src/test/scala/utils/kafka/utils/TestInfoUtils.scala @@ -1,4 +1,4 @@ -/** +/* * 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. @@ -14,8 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -// From: scala/kafka/utils/TestInfoUtils.scala package kafka.utils import java.lang.reflect.Method diff --git a/app/src/test/scala/utils/kafka/utils/TestUtils.scala b/app/src/test/scala/utils/kafka/utils/TestUtils.scala index 04636b2..2d33686 100644 --- a/app/src/test/scala/utils/kafka/utils/TestUtils.scala +++ b/app/src/test/scala/utils/kafka/utils/TestUtils.scala @@ -1,17 +1,19 @@ -/** - * 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 +/* + * 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 + * 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. + * 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. */ - -// From: scala/unit/kafka/utils/TestUtils.scala package kafka.utils import java.io._ @@ -144,9 +146,9 @@ object TestUtils extends Logging { val currentTestTimeMillis = System.currentTimeMillis() - /* Incorrect broker port which can used by kafka clients in tests. This port should not be used + /* Incorrect broker port which can used by kafka clients in tests. This port should not be used by any other service and hence we use a reserved port. */ - val IncorrectBrokerPort = 225 + val IncorrectBrokerPort = 225 // /** Port to use for unit tests that mock/don't require a real ZK server. */ // val MockZkPort = 1 @@ -2391,14 +2393,25 @@ object TestUtils extends Logging { // (out.toString, err.toString) // } // - def assertFutureExceptionTypeEquals(future: KafkaFuture[_], clazz: Class[_ <: Throwable], - expectedErrorMessage: Option[String] = None): Unit = { - val cause = assertThrows(classOf[ExecutionException], () => future.get()).getCause - assertTrue(clazz.isInstance(cause), "Expected an exception of type " + clazz.getName + "; got type " + - cause.getClass.getName) - expectedErrorMessage.foreach(message => assertTrue(cause.getMessage.contains(message), s"Received error message : ${cause.getMessage}" + - s" does not contain expected error message : $message")) - } + def assertFutureExceptionTypeEquals( + future: KafkaFuture[_], + clazz: Class[_ <: Throwable], + expectedErrorMessage: Option[String] = None + ): Unit = { + val cause = assertThrows(classOf[ExecutionException], () => future.get()).getCause + assertTrue( + clazz.isInstance(cause), + "Expected an exception of type " + clazz.getName + "; got type " + + cause.getClass.getName + ) + expectedErrorMessage.foreach(message => + assertTrue( + cause.getMessage.contains(message), + s"Received error message : ${cause.getMessage}" + + s" does not contain expected error message : $message" + ) + ) + } // // def assertBadConfigContainingMessage(props: Properties, expectedExceptionContainsText: String): Unit = { // try { diff --git a/java.header b/java.header new file mode 100644 index 0000000..45fd2d5 --- /dev/null +++ b/java.header @@ -0,0 +1,16 @@ +/* + * 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. + */